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   */
121  private static String getHostNameMinusDomain(final String hostname) {
122    if (InetAddresses.isInetAddress(hostname)) {
123      return hostname;
124    }
125    List<String> parts = Splitter.on('.').splitToList(hostname);
126    if (parts.size() == 0) {
127      return hostname;
128    }
129    Iterator<String> i = parts.iterator();
130    return i.next();
131  }
132
133  /**
134   * Retrieve an instance of ServerName. Callers should use the equals method to compare returned
135   * instances, though we may return a shared immutable object as an internal optimization.
136   */
137  public static ServerName valueOf(final String hostname, final int port, final long startCode) {
138    return INTERN_POOL.intern(new ServerName(hostname, port, startCode));
139  }
140
141  /**
142   * Retrieve an instance of ServerName. Callers should use the equals method to compare returned
143   * instances, though we may return a shared immutable object as an internal optimization.
144   */
145  public static ServerName valueOf(final String serverName) {
146    int firstSep = serverName.indexOf(SERVERNAME_SEPARATOR);
147    int lastSep = serverName.lastIndexOf(SERVERNAME_SEPARATOR);
148    String hostname = serverName.substring(0, firstSep);
149    int port = Integer.parseInt(serverName.substring(firstSep + 1, lastSep));
150    long startCode = Long.parseLong(serverName.substring(lastSep + 1));
151    return valueOf(hostname, port, startCode);
152  }
153
154  /**
155   * Retrieve an instance of ServerName. Callers should use the equals method to compare returned
156   * instances, though we may return a shared immutable object as an internal optimization.
157   */
158  public static ServerName valueOf(final String hostAndPort, final long startCode) {
159    return INTERN_POOL.intern(new ServerName(hostAndPort, startCode));
160  }
161
162  /**
163   * Retrieve an instance of {@link ServerName}. Callers should use the {@link #equals(Object)}
164   * method to compare returned instances, though we may return a shared immutable object as an
165   * internal optimization.
166   * @param address   the {@link Address} to use for getting the {@link ServerName}
167   * @param startCode the startcode to use for getting the {@link ServerName}
168   * @return the constructed {@link ServerName}
169   * @see #valueOf(String, int, long)
170   */
171  public static ServerName valueOf(final Address address, final long startCode) {
172    return valueOf(address.getHostname(), address.getPort(), startCode);
173  }
174
175  @Override
176  public String toString() {
177    return getServerName();
178  }
179
180  /**
181   * Return a SHORT version of {@link #toString()}, one that has the host only, minus the domain,
182   * and the port only -- no start code; the String is for us internally mostly tying threads to
183   * their server. Not for external use. It is lossy and will not work in in compares, etc.
184   */
185  public String toShortString() {
186    return Addressing.createHostAndPortStr(getHostNameMinusDomain(this.address.getHostname()),
187      this.address.getPort());
188  }
189
190  /**
191   * Return {@link #getServerName()} as bytes with a short-sized prefix with the {@link #VERSION} of
192   * this class.
193   */
194  public synchronized byte[] getVersionedBytes() {
195    if (this.bytes == null) {
196      this.bytes = Bytes.add(VERSION_BYTES, Bytes.toBytes(getServerName()));
197    }
198    return this.bytes;
199  }
200
201  public String getServerName() {
202    return serverName;
203  }
204
205  public String getHostname() {
206    return this.address.getHostname();
207  }
208
209  public String getHostnameLowerCase() {
210    return this.address.getHostname().toLowerCase(Locale.ROOT);
211  }
212
213  public int getPort() {
214    return this.address.getPort();
215  }
216
217  /**
218   * Return the start code.
219   * @deprecated Since 2.5.0, will be removed in 4.0.0. Use {@link #getStartCode()} instead.
220   */
221  @Deprecated
222  public long getStartcode() {
223    return startCode;
224  }
225
226  /** Return the start code. */
227  public long getStartCode() {
228    return startCode;
229  }
230
231  /**
232   * For internal use only.
233   * @param hostName  the name of the host to use
234   * @param port      the port on the host to use
235   * @param startCode the startcode to use for formatting
236   * @return Server name made of the concatenation of hostname, port and startcode formatted as
237   *         <code>&lt;hostname&gt; ',' &lt;port&gt; ',' &lt;startcode&gt;</code>
238   */
239  private static String getServerName(String hostName, int port, long startCode) {
240    return hostName.toLowerCase(Locale.ROOT) + SERVERNAME_SEPARATOR + port + SERVERNAME_SEPARATOR
241      + startCode;
242  }
243
244  public Address getAddress() {
245    return this.address;
246  }
247
248  @Override
249  public int compareTo(ServerName other) {
250    int compare;
251    if (other == null) {
252      return -1;
253    }
254    if (this.getHostname() == null) {
255      if (other.getHostname() != null) {
256        return 1;
257      }
258    } else {
259      if (other.getHostname() == null) {
260        return -1;
261      }
262      compare = this.getHostname().compareToIgnoreCase(other.getHostname());
263      if (compare != 0) {
264        return compare;
265      }
266    }
267    compare = this.getPort() - other.getPort();
268    if (compare != 0) {
269      return compare;
270    }
271    return Long.compare(this.getStartCode(), other.getStartCode());
272  }
273
274  @Override
275  public int hashCode() {
276    return getServerName().hashCode();
277  }
278
279  @Override
280  public boolean equals(Object o) {
281    if (this == o) {
282      return true;
283    }
284    if (o == null) {
285      return false;
286    }
287    if (!(o instanceof ServerName)) {
288      return false;
289    }
290    return this.compareTo((ServerName) o) == 0;
291  }
292
293  /**
294   * Compare two addresses
295   * @param left  the first server address to compare
296   * @param right the second server address to compare
297   * @return {@code true} if {@code left} and {@code right} have the same hostname and port.
298   */
299  public static boolean isSameAddress(final ServerName left, final ServerName right) {
300    return left.getAddress().equals(right.getAddress());
301  }
302
303  /**
304   * Use this method instantiating a {@link ServerName} from bytes gotten from a call to
305   * {@link #getVersionedBytes()}. Will take care of the case where bytes were written by an earlier
306   * version of hbase.
307   * @param versionedBytes Pass bytes gotten from a call to {@link #getVersionedBytes()}
308   * @return A ServerName instance.
309   * @see #getVersionedBytes()
310   */
311  public static ServerName parseVersionedServerName(final byte[] versionedBytes) {
312    // Version is a short.
313    short version = Bytes.toShort(versionedBytes);
314    if (version == VERSION) {
315      int length = versionedBytes.length - Bytes.SIZEOF_SHORT;
316      return valueOf(Bytes.toString(versionedBytes, Bytes.SIZEOF_SHORT, length));
317    }
318    // Presume the bytes were written with an old version of hbase and that the
319    // bytes are actually a String of the form "'<hostname>' ':' '<port>'".
320    return valueOf(Bytes.toString(versionedBytes), NON_STARTCODE);
321  }
322
323  /**
324   * Parse a ServerName from a string
325   * @param str Either an instance of {@link #toString()} or a "'&lt;hostname&gt;' ':'
326   *            '&lt;port&gt;'".
327   * @return A ServerName instance.
328   */
329  public static ServerName parseServerName(final String str) {
330    return SERVERNAME_PATTERN.matcher(str).matches() ? valueOf(str) : valueOf(str, NON_STARTCODE);
331  }
332
333  /** Returns true if the String follows the pattern of {@link #toString()}, false otherwise. */
334  public static boolean isFullServerName(final String str) {
335    if (str == null || str.isEmpty()) {
336      return false;
337    }
338    return SERVERNAME_PATTERN.matcher(str).matches();
339  }
340}