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