You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2011/10/28 00:20:03 UTC

svn commit: r1190068 - in /hbase/trunk: ./ src/main/java/org/apache/hadoop/hbase/ src/main/java/org/apache/hadoop/hbase/executor/ src/main/java/org/apache/hadoop/hbase/ipc/ src/main/java/org/apache/hadoop/hbase/master/ src/main/java/org/apache/hadoop/h...

Author: stack
Date: Thu Oct 27 22:20:02 2011
New Revision: 1190068

URL: http://svn.apache.org/viewvc?rev=1190068&view=rev
Log:
HBASE-4300 Start of new-version master fails if old master's znode is hanging around

Modified:
    hbase/trunk/CHANGES.txt
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/MasterAddressTracker.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/ServerName.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/executor/RegionTransitionData.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/ipc/HMasterRegionInterface.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeper.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServer.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/util/Addressing.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionServerTracker.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/RootRegionTracker.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/TestServerName.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestMasterAddressManager.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java

Modified: hbase/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hbase/trunk/CHANGES.txt?rev=1190068&r1=1190067&r2=1190068&view=diff
==============================================================================
--- hbase/trunk/CHANGES.txt (original)
+++ hbase/trunk/CHANGES.txt Thu Oct 27 22:20:02 2011
@@ -412,6 +412,8 @@ Release 0.92.0 - Unreleased
    HBASE-4388  Second start after migration from 90 to trunk crashes
    HBASE-4685  TestDistributedLogSplitting.testOrphanLogCreation failing because
                of ArithmeticException: / by zero.
+   HBASE-4300  Start of new-version master fails if old master's znode is
+               hanging around
 
   TESTS
    HBASE-4450  test for number of blocks read: to serve as baseline for expected

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java?rev=1190068&r1=1190067&r2=1190068&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java Thu Oct 27 22:20:02 2011
@@ -32,6 +32,7 @@ import java.util.Map;
 import java.util.TreeMap;
 
 import org.apache.hadoop.hbase.master.AssignmentManager.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.io.VersionedWritable;
 
 /**
@@ -223,12 +224,12 @@ public class ClusterStatus extends Versi
     out.writeUTF(hbaseVersion);
     out.writeInt(getServersSize());
     for (Map.Entry<ServerName, HServerLoad> e: this.liveServers.entrySet()) {
-      out.writeUTF(e.getKey().toString());
+      Bytes.writeByteArray(out, e.getKey().getVersionedBytes());
       e.getValue().write(out);
     }
     out.writeInt(deadServers.size());
     for (ServerName server: deadServers) {
-      out.writeUTF(server.toString());
+      Bytes.writeByteArray(out, server.getVersionedBytes());
     }
     out.writeInt(this.intransition.size());
     for (Map.Entry<String, RegionState> e: this.intransition.entrySet()) {
@@ -248,15 +249,15 @@ public class ClusterStatus extends Versi
     int count = in.readInt();
     this.liveServers = new HashMap<ServerName, HServerLoad>(count);
     for (int i = 0; i < count; i++) {
-      String str = in.readUTF();
+      byte [] versionedBytes = Bytes.readByteArray(in);
       HServerLoad hsl = new HServerLoad();
       hsl.readFields(in);
-      this.liveServers.put(new ServerName(str), hsl);
+      this.liveServers.put(ServerName.parseVersionedServerName(versionedBytes), hsl);
     }
     count = in.readInt();
     deadServers = new ArrayList<ServerName>(count);
     for (int i = 0; i < count; i++) {
-      deadServers.add(new ServerName(in.readUTF()));
+      deadServers.add(ServerName.parseVersionedServerName(Bytes.readByteArray(in)));
     }
     count = in.readInt();
     this.intransition = new TreeMap<String, RegionState>();

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/MasterAddressTracker.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/MasterAddressTracker.java?rev=1190068&r1=1190067&r2=1190068&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/MasterAddressTracker.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/MasterAddressTracker.java Thu Oct 27 22:20:02 2011
@@ -58,8 +58,7 @@ public class MasterAddressTracker extend
    * @return Server name or null if timed out.
    */
   public ServerName getMasterAddress() {
-    byte [] data = super.getData(false);
-    return data == null ? null : new ServerName(Bytes.toString(data));
+    return bytesToServerName(super.getData(false));
   }
 
   /**
@@ -70,4 +69,11 @@ public class MasterAddressTracker extend
     return super.getData(false) != null;
   }
 
+  /**
+   * @param bytes Byte array of {@link ServerName#toString()}
+   * @return A {@link ServerName} instance.
+   */
+  private ServerName bytesToServerName(final byte [] bytes) {
+    return bytes == null ? null: ServerName.parseVersionedServerName(bytes);
+  }
 }

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/ServerName.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/ServerName.java?rev=1190068&r1=1190067&r2=1190068&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/ServerName.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/ServerName.java Thu Oct 27 22:20:02 2011
@@ -20,6 +20,7 @@
 package org.apache.hadoop.hbase;
 
 import java.util.Collection;
+import java.util.regex.Pattern;
 
 import org.apache.hadoop.hbase.util.Addressing;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -37,19 +38,45 @@ import org.apache.hadoop.hbase.util.Byte
  * and the startcode for the regionserver is <code>1212121212</code>, then
  * the {@link #toString()} would be <code>example.org,1234,1212121212</code>.
  * 
+ * <p>You can obtain a versioned serialized form of this class by calling
+ * {@link #getVersionedBytes()}.  To deserialize, call {@link #parseVersionedServerName(byte[])}
+ * 
  * <p>Immutable.
  */
 public class ServerName implements Comparable<ServerName> {
   /**
+   * Version for this class.
+   * Its a short rather than a byte so I can for sure distinguish between this
+   * version of this class and the version previous to this which did not have
+   * a version.
+   */
+  private static final short VERSION = 0;
+  static final byte [] VERSION_BYTES = Bytes.toBytes(VERSION);
+
+  /**
+   * What to use if no startcode supplied.
+   */
+  public static final int NON_STARTCODE = -1;
+
+  /**
    * This character is used as separator between server hostname, port and
    * startcode.
    */
   public static final String SERVERNAME_SEPARATOR = ",";
 
+  public static Pattern SERVERNAME_PATTERN =
+    Pattern.compile(Addressing.VALID_HOSTNAME_REGEX_PREFIX +
+      SERVERNAME_SEPARATOR + Addressing.VALID_PORT_REGEX +
+      SERVERNAME_SEPARATOR + Addressing.VALID_PORT_REGEX + "$");
+
   private final String servername;
   private final String hostname;
   private final int port;
   private final long startcode;
+
+  /**
+   * Cached bytes of this ServerName instance.
+   */
   private byte [] bytes;
 
   public ServerName(final String hostname, final int port, final long startcode) {
@@ -64,10 +91,6 @@ public class ServerName implements Compa
       parseStartcode(serverName));
   }
 
-  public ServerName(final byte [] bytes) {
-    this(Bytes.toString(bytes));
-  }
-
   public ServerName(final String hostAndPort, final long startCode) {
     this(Addressing.parseHostname(hostAndPort),
       Addressing.parsePort(hostAndPort), startCode);
@@ -97,10 +120,13 @@ public class ServerName implements Compa
   }
 
   /**
-   * @return {@link #getServerName()} as bytes
+   * @return {@link #getServerName()} as bytes with a short-sized prefix with
+   * the {@link ServerName#VERSION} of this class.
    */
-  public synchronized byte [] getBytes() {
-    if (this.bytes == null) this.bytes = Bytes.toBytes(getServerName());
+  public synchronized byte [] getVersionedBytes() {
+    if (this.bytes == null) {
+      this.bytes = Bytes.add(VERSION_BYTES, Bytes.toBytes(getServerName()));
+    }
     return this.bytes;
   }
 
@@ -230,4 +256,34 @@ public class ServerName implements Compa
     return left.getHostname().equals(right.getHostname()) &&
       left.getPort() == right.getPort();
   }
+
+  /**
+   * Use this method instantiating a {@link ServerName} from bytes
+   * gotten from a call to {@link #getVersionedBytes()}.  Will take care of the
+   * case where bytes were written by an earlier version of hbase.
+   * @param versionedBytes Pass bytes gotten from a call to {@link #getVersionedBytes()}
+   * @return A ServerName instance.
+   * @see #getVersionedBytes()
+   */
+  public static ServerName parseVersionedServerName(final byte [] versionedBytes) {
+    // Version is a short.
+    short version = Bytes.toShort(versionedBytes);
+    if (version == VERSION) {
+      int length = versionedBytes.length - Bytes.SIZEOF_SHORT;
+      return new ServerName(Bytes.toString(versionedBytes, Bytes.SIZEOF_SHORT, length));
+    }
+    // Presume the bytes were written with an old version of hbase and that the
+    // bytes are actually a String of the form "'<hostname>' ':' '<port>'".
+    return new ServerName(Bytes.toString(versionedBytes), NON_STARTCODE);
+  }
+
+  /**
+   * @param str Either an instance of {@link ServerName#toString()} or a
+   * "'<hostname>' ':' '<port>'".
+   * @return A ServerName instance.
+   */
+  public static ServerName parseServerName(final String str) {
+    return SERVERNAME_PATTERN.matcher(str).matches()? new ServerName(str):
+      new ServerName(str, NON_STARTCODE);
+  }
 }

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/executor/RegionTransitionData.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/executor/RegionTransitionData.java?rev=1190068&r1=1190067&r2=1190068&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/executor/RegionTransitionData.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/executor/RegionTransitionData.java Thu Oct 27 22:20:02 2011
@@ -187,7 +187,8 @@ public class RegionTransitionData implem
     // remaining fields are optional so prefixed with boolean
     // the name of the regionserver sending the data
     if (in.readBoolean()) {
-      this.origin = new ServerName(in.readUTF());
+      byte [] versionedBytes = Bytes.readByteArray(in);
+      this.origin = ServerName.parseVersionedServerName(versionedBytes);
     }
     if (in.readBoolean()) {
       this.payload = Bytes.readByteArray(in);
@@ -201,8 +202,8 @@ public class RegionTransitionData implem
     Bytes.writeByteArray(out, regionName);
     // remaining fields are optional so prefixed with boolean
     out.writeBoolean(this.origin != null);
-    if(this.origin != null) {
-      out.writeUTF(this.origin.toString());
+    if (this.origin != null) {
+      Bytes.writeByteArray(out, this.origin.getVersionedBytes());
     }
     out.writeBoolean(this.payload != null);
     if (this.payload != null) {

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/ipc/HMasterRegionInterface.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/ipc/HMasterRegionInterface.java?rev=1190068&r1=1190067&r2=1190068&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/ipc/HMasterRegionInterface.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/ipc/HMasterRegionInterface.java Thu Oct 27 22:20:02 2011
@@ -55,7 +55,7 @@ public interface HMasterRegionInterface 
   throws IOException;
 
   /**
-   * @param sn {@link ServerName#getBytes()}
+   * @param sn {@link ServerName#getVersionedBytes()}
    * @param hsl Server load.
    * @throws IOException
    */
@@ -65,7 +65,7 @@ public interface HMasterRegionInterface 
   /**
    * Called by a region server to report a fatal error that is causing
    * it to abort.
-   * @param sn {@link ServerName#getBytes()}
+   * @param sn {@link ServerName#getVersionedBytes()}
    * @param errorMessage informative text to expose in the master logs and UI
    */
   public void reportRSFatalError(byte [] sn, String errorMessage);

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java?rev=1190068&r1=1190067&r2=1190068&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java Thu Oct 27 22:20:02 2011
@@ -145,7 +145,7 @@ class ActiveMasterManager extends ZooKee
       this.clusterHasActiveMaster.set(true);
       byte [] bytes =
         ZKUtil.getDataAndWatch(this.watcher, this.watcher.masterAddressZNode);
-      ServerName currentMaster = new ServerName(Bytes.toString(bytes));
+      ServerName currentMaster = ServerName.parseVersionedServerName(bytes);
       if (ServerName.isSameHostnameAndPort(currentMaster, this.sn)) {
         String msg = ("Current master has this master's address, " + currentMaster +
           "; master was restarted?  Waiting on znode to expire...");

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/HMaster.java?rev=1190068&r1=1190067&r2=1190068&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/HMaster.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/HMaster.java Thu Oct 27 22:20:02 2011
@@ -70,6 +70,7 @@ import org.apache.hadoop.hbase.master.ha
 import org.apache.hadoop.hbase.master.handler.TableDeleteFamilyHandler;
 import org.apache.hadoop.hbase.master.handler.TableModifyFamilyHandler;
 import org.apache.hadoop.hbase.master.metrics.MasterMetrics;
+import org.apache.hadoop.hbase.master.RegionPlan;
 import org.apache.hadoop.hbase.monitoring.MemoryBoundedLogMessageBuffer;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
@@ -723,7 +724,7 @@ implements HMasterInterface, HMasterRegi
   @Override
   public void regionServerReport(final byte [] sn, final HServerLoad hsl)
   throws IOException {
-    this.serverManager.regionServerReport(new ServerName(sn), hsl);
+    this.serverManager.regionServerReport(ServerName.parseVersionedServerName(sn), hsl);
     if (hsl != null && this.metrics != null) {
       // Up our metrics.
       this.metrics.incrementRequests(hsl.getTotalNumberOfRequests());
@@ -732,9 +733,8 @@ implements HMasterInterface, HMasterRegi
 
   @Override
   public void reportRSFatalError(byte [] sn, String errorText) {
-    ServerName serverName = new ServerName(sn);
-    String msg = "Region server " + serverName + " reported a fatal error:\n"
-        + errorText;
+    String msg = "Region server " + Bytes.toString(sn) +
+      " reported a fatal error:\n" + errorText;
     LOG.error(msg);
     rsFatals.add(msg);
   }

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java?rev=1190068&r1=1190067&r2=1190068&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java Thu Oct 27 22:20:02 2011
@@ -203,7 +203,11 @@ public class MasterFileSystem {
     }
     List<ServerName> serverNames = new ArrayList<ServerName>();
     for (FileStatus status : logFolders) {
-      ServerName serverName = new ServerName(status.getPath().getName());
+      String sn = status.getPath().getName();
+      // Is this old or new style servername?  If old style, it will be
+      // hostname, colon, and port.  If new style, it will be formatted as
+      // ServerName.toString.
+      ServerName serverName = ServerName.parseServerName(sn);
       if (!onlineServers.contains(serverName)) {
         LOG.info("Log folder " + status.getPath() + " doesn't belong " +
           "to a known region server, splitting");

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java?rev=1190068&r1=1190067&r2=1190068&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java Thu Oct 27 22:20:02 2011
@@ -248,7 +248,7 @@ public class ServerManager {
    * @deprecated Use {@link #getLoad(HServerAddress)}
    */
   public HServerLoad getLoad(final HServerAddress address) {
-    ServerName sn = new ServerName(address.toString(), -1);
+    ServerName sn = new ServerName(address.toString(), ServerName.NON_STARTCODE);
     ServerName actual =
       ServerName.findServerWithSameHostnamePort(this.getOnlineServersList(), sn);
     return actual == null? null: getLoad(actual);

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java?rev=1190068&r1=1190067&r2=1190068&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java Thu Oct 27 22:20:02 2011
@@ -784,7 +784,7 @@ public class HRegionServer implements HR
     // Why we do this?
     this.requestCount.set(0);
     try {
-      this.hbaseMaster.regionServerReport(this.serverNameFromMasterPOV.getBytes(), hsl);
+      this.hbaseMaster.regionServerReport(this.serverNameFromMasterPOV.getVersionedBytes(), hsl);
     } catch (IOException ioe) {
       if (ioe instanceof RemoteException) {
         ioe = ((RemoteException)ioe).unwrapRemoteException();
@@ -1613,7 +1613,7 @@ public class HRegionServer implements HR
       }
       if (hbaseMaster != null) {
         hbaseMaster.reportRSFatalError(
-            this.serverNameFromMasterPOV.getBytes(), msg);
+            this.serverNameFromMasterPOV.getVersionedBytes(), msg);
       }
     } catch (Throwable t) {
       LOG.warn("Unable to report fatal error to master", t);

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeper.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeper.java?rev=1190068&r1=1190067&r2=1190068&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeper.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeper.java Thu Oct 27 22:20:02 2011
@@ -282,7 +282,7 @@ public class ReplicationZookeeper {
     }
     List<ServerName> addresses = new ArrayList<ServerName>(children.size());
     for (String child : children) {
-      addresses.add(new ServerName(child));
+      addresses.add(ServerName.parseServerName(child));
     }
     return addresses;
   }

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServer.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServer.java?rev=1190068&r1=1190067&r2=1190068&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServer.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServer.java Thu Oct 27 22:20:02 2011
@@ -946,7 +946,7 @@ public class ThriftServer {
         value = startRowResult.getValue(HConstants.CATALOG_FAMILY,
                                         HConstants.SERVER_QUALIFIER);
         if (value != null && value.length > 0) {
-          ServerName sn = new ServerName(value);
+          ServerName sn = new ServerName(Bytes.toString(value), -1/*Any value works here for startcode*/);
           region.setServerName(Bytes.toBytes(sn.getHostname()));
           region.port = sn.getPort();
         }

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/util/Addressing.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/util/Addressing.java?rev=1190068&r1=1190067&r2=1190068&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/util/Addressing.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/util/Addressing.java Thu Oct 27 22:20:02 2011
@@ -25,6 +25,21 @@ import java.net.InetSocketAddress;
  * Utility for network addresses, resolving and naming.
  */
 public class Addressing {
+  /**
+   * Regex for RFC952 hostname matching.  Does not have a '$" on the end.  Can
+   * be used as prefix on a larger regex.
+   * @see http://stackoverflow.com/questions/106179/regular-expression-to-match-hostname-or-ip-address
+   */
+  public static final String VALID_HOSTNAME_REGEX_PREFIX =
+      "^(([a-zA-Z]|[a-zA-Z][a-zA-Z0-9\\-]*[a-zA-Z0-9])\\.)*([A-Za-z]|[A-Za-z][A-Za-z0-9\\-]*[A-Za-z0-9])";
+  /**
+   * Regex for RFC952 hostname matching.
+   * @see http://stackoverflow.com/questions/106179/regular-expression-to-match-hostname-or-ip-address
+   */
+  public static final String VALID_HOSTNAME_REGEX =
+    VALID_HOSTNAME_REGEX_PREFIX + "$";
+
+  public static final String VALID_PORT_REGEX = "[\\d]+";
   public static final String HOSTNAME_PORT_SEPARATOR = ":";
 
   /**

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionServerTracker.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionServerTracker.java?rev=1190068&r1=1190067&r2=1190068&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionServerTracker.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionServerTracker.java Thu Oct 27 22:20:02 2011
@@ -74,7 +74,7 @@ public class RegionServerTracker extends
     synchronized(this.regionServers) {
       this.regionServers.clear();
       for (String n: servers) {
-        ServerName sn = new ServerName(ZKUtil.getNodeName(n));
+        ServerName sn = ServerName.parseServerName(ZKUtil.getNodeName(n));
         this.regionServers.add(sn);
       }
     }
@@ -92,7 +92,7 @@ public class RegionServerTracker extends
       String serverName = ZKUtil.getNodeName(path);
       LOG.info("RegionServer ephemeral node deleted, processing expiration [" +
         serverName + "]");
-      ServerName sn = new ServerName(serverName);
+      ServerName sn = ServerName.parseServerName(serverName);
       if (!serverManager.isServerOnline(sn)) {
         LOG.info(serverName.toString() + " is not online");
         return;
@@ -126,4 +126,4 @@ public class RegionServerTracker extends
       return new ArrayList<ServerName>(this.regionServers);
     }
   }
-}
\ No newline at end of file
+}

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/RootRegionTracker.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/RootRegionTracker.java?rev=1190068&r1=1190067&r2=1190068&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/RootRegionTracker.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/RootRegionTracker.java Thu Oct 27 22:20:02 2011
@@ -97,7 +97,7 @@ public class RootRegionTracker extends Z
     int index = str.indexOf(ServerName.SERVERNAME_SEPARATOR);
     if (index != -1) {
       // Presume its ServerName.toString() format.
-      return new ServerName(str);
+      return ServerName.parseServerName(str);
     }
     // Presume it a hostname:port format.
     String hostname = Addressing.parseHostname(str);

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/TestServerName.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/TestServerName.java?rev=1190068&r1=1190067&r2=1190068&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/TestServerName.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/TestServerName.java Thu Oct 27 22:20:02 2011
@@ -1,6 +1,4 @@
 /**
- * Copyright 2011 The Apache Software Foundation
- *
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -19,15 +17,42 @@
  */
 package org.apache.hadoop.hbase;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotSame;
+import static org.junit.Assert.assertTrue;
 
-import java.util.HashSet;
-import java.util.Set;
+import java.util.regex.Pattern;
 
+import org.apache.hadoop.hbase.util.Addressing;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.Test;
 
 public class TestServerName {
   @Test
+  public void testRegexPatterns() {
+    assertTrue(Pattern.matches(Addressing.VALID_PORT_REGEX, "123"));
+    assertFalse(Pattern.matches(Addressing.VALID_PORT_REGEX, ""));
+    assertTrue(Pattern.matches(Addressing.VALID_HOSTNAME_REGEX, "example.org"));
+    assertTrue(Pattern.matches(Addressing.VALID_HOSTNAME_REGEX,
+      "www1.example.org"));
+    assertTrue(ServerName.SERVERNAME_PATTERN.matcher("www1.example.org,1234,567").matches());
+  }
+
+  @Test public void testParseOfBytes() {
+    final String snStr = "www.example.org,1234,5678";
+    ServerName sn = new ServerName(snStr);
+    byte [] versionedBytes = sn.getVersionedBytes();
+    assertEquals(snStr, ServerName.parseVersionedServerName(versionedBytes).toString());
+    final String hostnamePortStr = "www.example.org:1234";
+    byte [] bytes = Bytes.toBytes(hostnamePortStr);
+    String expecting =
+      hostnamePortStr.replace(":", ServerName.SERVERNAME_SEPARATOR) +
+      ServerName.SERVERNAME_SEPARATOR + ServerName.NON_STARTCODE;
+    assertEquals(expecting, ServerName.parseVersionedServerName(bytes).toString());
+  }
+
+  @Test
   public void testServerName() {
     ServerName sn = new ServerName("www.example.org", 1234, 5678);
     ServerName sn2 = new ServerName("www.example.org", 1234, 5678);

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestMasterAddressManager.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestMasterAddressManager.java?rev=1190068&r1=1190067&r2=1190068&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestMasterAddressManager.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestMasterAddressManager.java Thu Oct 27 22:20:02 2011
@@ -77,7 +77,7 @@ public class TestMasterAddressManager {
     int port = 1234;
     ServerName sn = new ServerName(host, port, System.currentTimeMillis());
     LOG.info("Creating master node");
-    ZKUtil.createEphemeralNodeAndWatch(zk, zk.masterAddressZNode, sn.getBytes());
+    ZKUtil.createEphemeralNodeAndWatch(zk, zk.masterAddressZNode, sn.getVersionedBytes());
 
     // Wait for the node to be created
     LOG.info("Waiting for master address manager to be notified");

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java?rev=1190068&r1=1190067&r2=1190068&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java Thu Oct 27 22:20:02 2011
@@ -405,7 +405,7 @@ public class TestSplitTransactionOnClust
       HRegionServer hrs = getOtherRegionServer(cluster, metaRegionServer);
       LOG.info("Moving " + hri.getRegionNameAsString() + " to " +
         hrs.getServerName() + "; metaServerIndex=" + metaServerIndex);
-      admin.move(hri.getEncodedNameAsBytes(), hrs.getServerName().getBytes());
+      admin.move(hri.getEncodedNameAsBytes(), hrs.getServerName().getVersionedBytes());
     }
     // Wait till table region is up on the server that is NOT carrying .META..
     while (true) {