001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase;
019
020import java.io.Serializable;
021import java.util.ArrayList;
022import java.util.Iterator;
023import java.util.List;
024import java.util.Locale;
025import java.util.regex.Pattern;
026import org.apache.hadoop.hbase.net.Address;
027import org.apache.hadoop.hbase.util.Addressing;
028import org.apache.hadoop.hbase.util.Bytes;
029import org.apache.yetus.audience.InterfaceAudience;
030
031import org.apache.hbase.thirdparty.com.google.common.base.Splitter;
032import org.apache.hbase.thirdparty.com.google.common.collect.Interner;
033import org.apache.hbase.thirdparty.com.google.common.collect.Interners;
034import org.apache.hbase.thirdparty.com.google.common.net.InetAddresses;
035
036/**
037 * Name of a particular incarnation of an HBase Server. A {@link ServerName} is used uniquely
038 * identifying a server instance in a cluster and is made of the combination of hostname, port, and
039 * startcode. The startcode distinguishes restarted servers on same hostname and port (startcode is
040 * usually timestamp of server startup). The {@link #toString()} format of ServerName is safe to use
041 * in the filesystem and as znode name up in ZooKeeper. Its format is:
042 * <code>&lt;hostname&gt; '{@link #SERVERNAME_SEPARATOR}' &lt;port&gt;
043 * '{@link #SERVERNAME_SEPARATOR}' &lt;startcode&gt;</code>. For example, if hostname is
044 * <code>www.example.org</code>, port is <code>1234</code>, and the startcode for the regionserver
045 * is <code>1212121212</code>, then the {@link #toString()} would be
046 * <code>www.example.org,1234,1212121212</code>.
047 * <p>
048 * You can obtain a versioned serialized form of this class by calling {@link #getVersionedBytes()}.
049 * To deserialize, call {@link #parseVersionedServerName(byte[])}.
050 * <p>
051 * Use {@link #getAddress()} to obtain the Server hostname + port (Endpoint/Socket Address).
052 * <p>
053 * Immutable.
054 */
055@InterfaceAudience.Public
056public class ServerName implements Comparable<ServerName>, Serializable {
057  private static final long serialVersionUID = 1367463982557264981L;
058
059  /**
060   * Version for this class. Its a short rather than a byte so I can for sure distinguish between
061   * this version of this class and the version previous to this which did not have a version.
062   */
063  private static final short VERSION = 0;
064  static final byte[] VERSION_BYTES = Bytes.toBytes(VERSION);
065
066  /**
067   * What to use if no startcode supplied.
068   */
069  public static final int NON_STARTCODE = -1;
070
071  /**
072   * This character is used as separator between server hostname, port and startcode.
073   */
074  public static final String SERVERNAME_SEPARATOR = ",";
075
076  public static final Pattern SERVERNAME_PATTERN =
077    Pattern.compile("[^" + SERVERNAME_SEPARATOR + "]+" + SERVERNAME_SEPARATOR
078      + Addressing.VALID_PORT_REGEX + SERVERNAME_SEPARATOR + Addressing.VALID_PORT_REGEX + "$");
079
080  /**
081   * What to use if server name is unknown.
082   */
083  public static final String UNKNOWN_SERVERNAME = "#unknown#";
084
085  private final String serverName;
086  private final long startCode;
087  private transient Address address;
088
089  /**
090   * Cached versioned bytes of this ServerName instance.
091   * @see #getVersionedBytes()
092   */
093  private byte[] bytes;
094  public static final List<ServerName> EMPTY_SERVER_LIST = new ArrayList<>(0);
095
096  /**
097   * Intern ServerNames. The Set of ServerNames is mostly-fixed changing slowly as Servers restart.
098   * Rather than create a new instance everytime, try and return existing instance if there is one.
099   */
100  private static final Interner<ServerName> INTERN_POOL = Interners.newWeakInterner();
101
102  protected ServerName(final String hostname, final int port, final long startCode) {
103    this(Address.fromParts(hostname, port), startCode);
104  }
105
106  private ServerName(final Address address, final long startCode) {
107    // Use HostAndPort to host port and hostname. Does validation and can do ipv6
108    this.address = address;
109    this.startCode = startCode;
110    this.serverName = getServerName(this.address.getHostname(), this.address.getPort(), startCode);
111  }
112
113  private ServerName(final String hostAndPort, final long startCode) {
114    this(Address.fromString(hostAndPort), startCode);
115  }
116
117  /**
118   * @param hostname the hostname string to get the actual hostname from
119   * @return hostname minus the domain, if there is one (will do pass-through on ip addresses)
120   * @deprecated Since 2.0. This is for internal use only.
121   */
122  @Deprecated
123  // Make this private in hbase-3.0.
124  static String getHostNameMinusDomain(final String hostname) {
125    if (InetAddresses.isInetAddress(hostname)) {
126      return hostname;
127    }
128    List<String> parts = Splitter.on('.').splitToList(hostname);
129    if (parts.size() == 0) {
130      return hostname;
131    }
132    Iterator<String> i = parts.iterator();
133    return i.next();
134  }
135
136  /**
137   * @deprecated Since 2.0. Use {@link #valueOf(String)}
138   */
139  @Deprecated
140  // This is unused. Get rid of it.
141  public static String parseHostname(final String serverName) {
142    if (serverName == null || serverName.length() <= 0) {
143      throw new IllegalArgumentException("Passed hostname is null or empty");
144    }
145    if (!Character.isLetterOrDigit(serverName.charAt(0))) {
146      throw new IllegalArgumentException("Bad passed hostname, serverName=" + serverName);
147    }
148    int index = serverName.indexOf(SERVERNAME_SEPARATOR);
149    return serverName.substring(0, index);
150  }
151
152  /**
153   * @deprecated Since 2.0. Use {@link #valueOf(String)}
154   */
155  @Deprecated
156  // This is unused. Get rid of it.
157  public static int parsePort(final String serverName) {
158    String[] split = serverName.split(SERVERNAME_SEPARATOR);
159    return Integer.parseInt(split[1]);
160  }
161
162  /**
163   * @deprecated Since 2.0. Use {@link #valueOf(String)}
164   */
165  @Deprecated
166  // This is unused. Get rid of it.
167  public static long parseStartcode(final String serverName) {
168    int index = serverName.lastIndexOf(SERVERNAME_SEPARATOR);
169    return Long.parseLong(serverName.substring(index + 1));
170  }
171
172  /**
173   * Retrieve an instance of ServerName. Callers should use the equals method to compare returned
174   * instances, though we may return a shared immutable object as an internal optimization.
175   */
176  public static ServerName valueOf(final String hostname, final int port, final long startCode) {
177    return INTERN_POOL.intern(new ServerName(hostname, port, startCode));
178  }
179
180  /**
181   * Retrieve an instance of ServerName. Callers should use the equals method to compare returned
182   * instances, though we may return a shared immutable object as an internal optimization.
183   */
184  public static ServerName valueOf(final String serverName) {
185    int firstSep = serverName.indexOf(SERVERNAME_SEPARATOR);
186    int lastSep = serverName.lastIndexOf(SERVERNAME_SEPARATOR);
187    String hostname = serverName.substring(0, firstSep);
188    int port = Integer.parseInt(serverName.substring(firstSep + 1, lastSep));
189    long startCode = Long.parseLong(serverName.substring(lastSep + 1));
190    return valueOf(hostname, port, startCode);
191  }
192
193  /**
194   * Retrieve an instance of ServerName. Callers should use the equals method to compare returned
195   * instances, though we may return a shared immutable object as an internal optimization.
196   */
197  public static ServerName valueOf(final String hostAndPort, final long startCode) {
198    return INTERN_POOL.intern(new ServerName(hostAndPort, startCode));
199  }
200
201  /**
202   * Retrieve an instance of {@link ServerName}. Callers should use the {@link #equals(Object)}
203   * method to compare returned instances, though we may return a shared immutable object as an
204   * internal optimization.
205   * @param address   the {@link Address} to use for getting the {@link ServerName}
206   * @param startCode the startcode to use for getting the {@link ServerName}
207   * @return the constructed {@link ServerName}
208   * @see #valueOf(String, int, long)
209   */
210  public static ServerName valueOf(final Address address, final long startCode) {
211    return valueOf(address.getHostname(), address.getPort(), startCode);
212  }
213
214  @Override
215  public String toString() {
216    return getServerName();
217  }
218
219  /**
220   * Return a SHORT version of {@link #toString()}, one that has the host only, minus the domain,
221   * and the port only -- no start code; the String is for us internally mostly tying threads to
222   * their server. Not for external use. It is lossy and will not work in in compares, etc.
223   */
224  public String toShortString() {
225    return Addressing.createHostAndPortStr(getHostNameMinusDomain(this.address.getHostname()),
226      this.address.getPort());
227  }
228
229  /**
230   * Return {@link #getServerName()} as bytes with a short-sized prefix with the {@link #VERSION} of
231   * this class.
232   */
233  public synchronized byte[] getVersionedBytes() {
234    if (this.bytes == null) {
235      this.bytes = Bytes.add(VERSION_BYTES, Bytes.toBytes(getServerName()));
236    }
237    return this.bytes;
238  }
239
240  public String getServerName() {
241    return serverName;
242  }
243
244  public String getHostname() {
245    return this.address.getHostname();
246  }
247
248  public String getHostnameLowerCase() {
249    return this.address.getHostname().toLowerCase(Locale.ROOT);
250  }
251
252  public int getPort() {
253    return this.address.getPort();
254  }
255
256  /**
257   * Return the start code.
258   * @deprecated Since 2.5.0, will be removed in 4.0.0. Use {@link #getStartCode()} instead.
259   */
260  @Deprecated
261  public long getStartcode() {
262    return startCode;
263  }
264
265  /** Return the start code. */
266  public long getStartCode() {
267    return startCode;
268  }
269
270  /**
271   * For internal use only.
272   * @param hostName  the name of the host to use
273   * @param port      the port on the host to use
274   * @param startCode the startcode to use for formatting
275   * @return Server name made of the concatenation of hostname, port and startcode formatted as
276   *         <code>&lt;hostname&gt; ',' &lt;port&gt; ',' &lt;startcode&gt;</code>
277   * @deprecated Since 2.0. Use {@link ServerName#valueOf(String, int, long)} instead.
278   */
279  @Deprecated
280  // TODO: Make this private in hbase-3.0.
281  static String getServerName(String hostName, int port, long startCode) {
282    return hostName.toLowerCase(Locale.ROOT) + SERVERNAME_SEPARATOR + port + SERVERNAME_SEPARATOR
283      + startCode;
284  }
285
286  /**
287   * @param hostAndPort String in form of &lt;hostname&gt; ':' &lt;port&gt;
288   * @param startcode   the startcode to use
289   * @return Server name made of the concatenation of hostname, port and startcode formatted as
290   *         <code>&lt;hostname&gt; ',' &lt;port&gt; ',' &lt;startcode&gt;</code>
291   * @deprecated Since 2.0. Use {@link ServerName#valueOf(String, long)} instead.
292   */
293  @Deprecated
294  public static String getServerName(final String hostAndPort, final long startcode) {
295    int index = hostAndPort.indexOf(':');
296    if (index <= 0) {
297      throw new IllegalArgumentException("Expected <hostname> ':' <port>");
298    }
299    return getServerName(hostAndPort.substring(0, index),
300      Integer.parseInt(hostAndPort.substring(index + 1)), startcode);
301  }
302
303  /**
304   * @return Hostname and port formatted as described at
305   *         {@link Addressing#createHostAndPortStr(String, int)}
306   * @deprecated Since 2.0. Use {@link #getAddress()} instead.
307   */
308  @Deprecated
309  public String getHostAndPort() {
310    return this.address.toString();
311  }
312
313  public Address getAddress() {
314    return this.address;
315  }
316
317  /**
318   * @param serverName ServerName in form specified by {@link #getServerName()}
319   * @return The server start code parsed from <code>servername</code>
320   * @deprecated Since 2.0. Use instance of ServerName to pull out start code.
321   */
322  @Deprecated
323  public static long getServerStartcodeFromServerName(final String serverName) {
324    int index = serverName.lastIndexOf(SERVERNAME_SEPARATOR);
325    return Long.parseLong(serverName.substring(index + 1));
326  }
327
328  /**
329   * Utility method to excise the start code from a server name
330   * @param inServerName full server name
331   * @return server name less its start code
332   * @deprecated Since 2.0. Use {@link #getAddress()}
333   */
334  @Deprecated
335  public static String getServerNameLessStartCode(String inServerName) {
336    if (inServerName != null && inServerName.length() > 0) {
337      int index = inServerName.lastIndexOf(SERVERNAME_SEPARATOR);
338      if (index > 0) {
339        return inServerName.substring(0, index);
340      }
341    }
342    return inServerName;
343  }
344
345  @Override
346  public int compareTo(ServerName other) {
347    int compare;
348    if (other == null) {
349      return -1;
350    }
351    if (this.getHostname() == null) {
352      if (other.getHostname() != null) {
353        return 1;
354      }
355    } else {
356      if (other.getHostname() == null) {
357        return -1;
358      }
359      compare = this.getHostname().compareToIgnoreCase(other.getHostname());
360      if (compare != 0) {
361        return compare;
362      }
363    }
364    compare = this.getPort() - other.getPort();
365    if (compare != 0) {
366      return compare;
367    }
368    return Long.compare(this.getStartCode(), other.getStartCode());
369  }
370
371  @Override
372  public int hashCode() {
373    return getServerName().hashCode();
374  }
375
376  @Override
377  public boolean equals(Object o) {
378    if (this == o) {
379      return true;
380    }
381    if (o == null) {
382      return false;
383    }
384    if (!(o instanceof ServerName)) {
385      return false;
386    }
387    return this.compareTo((ServerName) o) == 0;
388  }
389
390  /**
391   * Compare two addresses
392   * @param left  the first server address to compare
393   * @param right the second server address to compare
394   * @return {@code true} if {@code left} and {@code right} have the same hostname and port.
395   */
396  public static boolean isSameAddress(final ServerName left, final ServerName right) {
397    return left.getAddress().equals(right.getAddress());
398  }
399
400  /**
401   * Use this method instantiating a {@link ServerName} from bytes gotten from a call to
402   * {@link #getVersionedBytes()}. Will take care of the case where bytes were written by an earlier
403   * version of hbase.
404   * @param versionedBytes Pass bytes gotten from a call to {@link #getVersionedBytes()}
405   * @return A ServerName instance.
406   * @see #getVersionedBytes()
407   */
408  public static ServerName parseVersionedServerName(final byte[] versionedBytes) {
409    // Version is a short.
410    short version = Bytes.toShort(versionedBytes);
411    if (version == VERSION) {
412      int length = versionedBytes.length - Bytes.SIZEOF_SHORT;
413      return valueOf(Bytes.toString(versionedBytes, Bytes.SIZEOF_SHORT, length));
414    }
415    // Presume the bytes were written with an old version of hbase and that the
416    // bytes are actually a String of the form "'<hostname>' ':' '<port>'".
417    return valueOf(Bytes.toString(versionedBytes), NON_STARTCODE);
418  }
419
420  /**
421   * Parse a ServerName from a string
422   * @param str Either an instance of {@link #toString()} or a "'&lt;hostname&gt;' ':'
423   *            '&lt;port&gt;'".
424   * @return A ServerName instance.
425   */
426  public static ServerName parseServerName(final String str) {
427    return SERVERNAME_PATTERN.matcher(str).matches() ? valueOf(str) : valueOf(str, NON_STARTCODE);
428  }
429
430  /** Returns true if the String follows the pattern of {@link #toString()}, false otherwise. */
431  public static boolean isFullServerName(final String str) {
432    if (str == null || str.isEmpty()) {
433      return false;
434    }
435    return SERVERNAME_PATTERN.matcher(str).matches();
436  }
437}