You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by nk...@apache.org on 2013/11/20 21:06:17 UTC

svn commit: r1543923 [1/2] - in /hbase/branches/0.96: hbase-client/src/main/java/org/apache/hadoop/hbase/ hbase-client/src/main/java/org/apache/hadoop/hbase/client/ hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/ hbase-client/src/main/ja...

Author: nkeywal
Date: Wed Nov 20 20:06:16 2013
New Revision: 1543923

URL: http://svn.apache.org/r1543923
Log:
HBASE-10012 Hide ServerName constructor

Modified:
    hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
    hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerName.java
    hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
    hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/RegionMovedException.java
    hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
    hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
    hbase/branches/0.96/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
    hbase/branches/0.96/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java
    hbase/branches/0.96/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RollingBatchRestartRsAction.java
    hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
    hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
    hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java
    hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
    hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredNodeLoadBalancer.java
    hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/migration/NamespaceUpgrade.java
    hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
    hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/DrainingServerTracker.java
    hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/TestDrainingServer.java
    hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHRegionLocation.java
    hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/TestSerialization.java
    hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerName.java
    hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestCatalogTracker.java
    hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaReaderEditorNoCluster.java
    hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java
    hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaScanner.java
    hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestDelayedRpc.java
    hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestProtoBufRpc.java
    hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesSplitRecovery.java
    hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java
    hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManager.java
    hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java
    hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
    hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClusterStatusPublisher.java
    hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDeadServer.java
    hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystem.java
    hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
    hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterStatusServlet.java
    hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java
    hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java
    hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java
    hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestTableLockManager.java
    hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerTestBase.java
    hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestBaseLoadBalancer.java
    hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredNodeAssignmentHelper.java
    hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancer.java
    hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
    hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java
    hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
    hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMasterAddressManager.java
    hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSStatusServlet.java
    hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java
    hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
    hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java
    hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
    hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java
    hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java
    hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
    hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java
    hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockServer.java
    hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java
    hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperNodeTracker.java

Modified: hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java?rev=1543923&r1=1543922&r2=1543923&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java (original)
+++ hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java Wed Nov 20 20:06:16 2013
@@ -1041,7 +1041,7 @@ public class HRegionInfo implements Comp
     cell = r.getColumnLatestCell(HConstants.CATALOG_FAMILY,
       HConstants.STARTCODE_QUALIFIER);
     if (cell == null || cell.getValueLength() == 0) return null;
-    return new ServerName(hostAndPort,
+    return ServerName.valueOf(hostAndPort,
         Bytes.toLong(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()));
   }
 

Modified: hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerName.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerName.java?rev=1543923&r1=1543922&r2=1543923&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerName.java (original)
+++ hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerName.java Wed Nov 20 20:06:16 2013
@@ -94,7 +94,7 @@ public class ServerName implements Compa
   private byte [] bytes;
   public static final List<ServerName> EMPTY_SERVER_LIST = new ArrayList<ServerName>(0);
 
-  public ServerName(final String hostname, final int port, final long startcode) {
+  private ServerName(final String hostname, final int port, final long startcode) {
     // Drop the domain is there is one; no need of it in a local cluster.  With it, we get long
     // unwieldy names.
     this.hostnameOnly = hostname;
@@ -114,12 +114,12 @@ public class ServerName implements Compa
     return parts[0];
   }
 
-  public ServerName(final String serverName) {
+  private ServerName(final String serverName) {
     this(parseHostname(serverName), parsePort(serverName),
       parseStartcode(serverName));
   }
 
-  public ServerName(final String hostAndPort, final long startCode) {
+  private ServerName(final String hostAndPort, final long startCode) {
     this(Addressing.parseHostname(hostAndPort),
       Addressing.parsePort(hostAndPort), startCode);
   }
@@ -145,6 +145,18 @@ public class ServerName implements Compa
     return Long.parseLong(serverName.substring(index + 1));
   }
 
+  public static ServerName valueOf(final String hostAndPort, final long startCode) {
+    return new ServerName(hostAndPort, startCode);
+  }
+
+  public static ServerName valueOf(final String serverName) {
+    return new ServerName(serverName);
+  }
+
+  public static ServerName valueOf(final String hostname, final int port, final long startcode) {
+    return new ServerName(hostname, port, startcode);
+  }
+
   @Override
   public String toString() {
     return getServerName();
@@ -299,11 +311,11 @@ public class ServerName implements Compa
     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));
+      return valueOf(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);
+    return valueOf(Bytes.toString(versionedBytes), NON_STARTCODE);
   }
 
   /**
@@ -312,8 +324,8 @@ public class ServerName implements Compa
    * @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);
+    return SERVERNAME_PATTERN.matcher(str).matches()? valueOf(str) :
+        valueOf(str, NON_STARTCODE);
   }
 
 
@@ -344,7 +356,7 @@ public class ServerName implements Compa
         MetaRegionServer rss =
           MetaRegionServer.PARSER.parseFrom(data, prefixLen, data.length - prefixLen);
         org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName sn = rss.getServer();
-        return new ServerName(sn.getHostName(), sn.getPort(), sn.getStartCode());
+        return valueOf(sn.getHostName(), sn.getPort(), sn.getStartCode());
       } catch (InvalidProtocolBufferException e) {
         // A failed parse of the znode is pretty catastrophic. Rather than loop
         // retrying hoping the bad bytes will changes, and rather than change
@@ -366,6 +378,6 @@ public class ServerName implements Compa
     // Presume it a hostname:port format.
     String hostname = Addressing.parseHostname(str);
     int port = Addressing.parsePort(str);
-    return new ServerName(hostname, port, -1L);
+    return valueOf(hostname, port, -1L);
   }
 }

Modified: hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java?rev=1543923&r1=1543922&r2=1543923&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java (original)
+++ hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java Wed Nov 20 20:06:16 2013
@@ -1324,7 +1324,7 @@ public class HBaseAdmin implements Abort
         if (pair == null || pair.getFirst() == null) {
           throw new UnknownRegionException(Bytes.toStringBinary(regionname));
         } else {
-          closeRegion(new ServerName(serverName), pair.getFirst());
+          closeRegion(ServerName.valueOf(serverName), pair.getFirst());
         }
       } else {
         Pair<HRegionInfo, ServerName> pair = MetaReader.getRegion(ct, regionname);
@@ -1368,7 +1368,7 @@ public class HBaseAdmin implements Abort
       throw new IllegalArgumentException(
           "The servername cannot be null or empty.");
     }
-    ServerName sn = new ServerName(serverName);
+    ServerName sn = ServerName.valueOf(serverName);
     AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
     // Close the region without updating zk state.
     CloseRegionRequest request =
@@ -2126,7 +2126,7 @@ public class HBaseAdmin implements Abort
     String hostname = Addressing.parseHostname(hostnamePort);
     int port = Addressing.parsePort(hostnamePort);
     AdminService.BlockingInterface admin =
-      this.connection.getAdmin(new ServerName(hostname, port, 0));
+      this.connection.getAdmin(ServerName.valueOf(hostname, port, 0));
     StopServerRequest request = RequestConverter.buildStopServerRequest(
       "Called by admin client " + this.connection.toString());
     try {
@@ -2426,7 +2426,7 @@ public class HBaseAdmin implements Abort
    */
  public synchronized  byte[][] rollHLogWriter(String serverName)
       throws IOException, FailedLogCloseException {
-    ServerName sn = new ServerName(serverName);
+    ServerName sn = ServerName.valueOf(serverName);
     AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
     RollWALWriterRequest request = RequestConverter.buildRollWALWriterRequest();
     try {

Modified: hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/RegionMovedException.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/RegionMovedException.java?rev=1543923&r1=1543922&r2=1543923&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/RegionMovedException.java (original)
+++ hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/RegionMovedException.java Wed Nov 20 20:06:16 2013
@@ -24,7 +24,6 @@ import org.apache.hadoop.classification.
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.NotServingRegionException;
 import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.ipc.RemoteException;
 
 /**
  * Subclass if the server knows the region is now on another server.
@@ -63,7 +62,7 @@ public class RegionMovedException extend
   }
 
   public ServerName getServerName(){
-    return new ServerName(hostname, port, startCode);
+    return ServerName.valueOf(hostname, port, startCode);
   }
 
   public long getLocationSeqNum() {

Modified: hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java?rev=1543923&r1=1543922&r2=1543923&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java (original)
+++ hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java Wed Nov 20 20:06:16 2013
@@ -318,7 +318,7 @@ public final class ProtobufUtil {
     if (proto.hasStartCode()) {
       startCode = proto.getStartCode();
     }
-    return new ServerName(hostName, port, startCode);
+    return ServerName.valueOf(hostName, port, startCode);
   }
 
   /**

Modified: hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java?rev=1543923&r1=1543922&r2=1543923&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java (original)
+++ hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java Wed Nov 20 20:06:16 2013
@@ -984,7 +984,7 @@ public final class RequestConverter {
       buildRegionSpecifier(RegionSpecifierType.ENCODED_REGION_NAME,encodedRegionName));
     if (destServerName != null) {
       builder.setDestServerName(
-        ProtobufUtil.toServerName(new ServerName(Bytes.toString(destServerName))));
+        ProtobufUtil.toServerName(ServerName.valueOf(Bytes.toString(destServerName))));
     }
     return builder.build();
   }

Modified: hbase/branches/0.96/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java?rev=1543923&r1=1543922&r2=1543923&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java (original)
+++ hbase/branches/0.96/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java Wed Nov 20 20:06:16 2013
@@ -58,8 +58,8 @@ public class TestAsyncProcess {
   private static final byte[] FAILS = "FAILS".getBytes();
   private static final Configuration conf = new Configuration();
 
-  private static ServerName sn = new ServerName("localhost:10,1254");
-  private static ServerName sn2 = new ServerName("localhost:140,12540");
+  private static ServerName sn = ServerName.valueOf("localhost:10,1254");
+  private static ServerName sn2 = ServerName.valueOf("localhost:140,12540");
   private static HRegionInfo hri1 =
       new HRegionInfo(DUMMY_TABLE, DUMMY_BYTES_1, DUMMY_BYTES_2, false, 1);
   private static HRegionInfo hri2 =

Modified: hbase/branches/0.96/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java?rev=1543923&r1=1543922&r2=1543923&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java (original)
+++ hbase/branches/0.96/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java Wed Nov 20 20:06:16 2013
@@ -95,7 +95,7 @@ public class TestClientNoCluster extends
   private static final Log LOG = LogFactory.getLog(TestClientNoCluster.class);
   private Configuration conf;
   public static final ServerName META_SERVERNAME =
-    new ServerName("meta.example.org", 60010, 12345);
+      ServerName.valueOf("meta.example.org", 60010, 12345);
 
   @Before
   public void setUp() throws Exception {
@@ -651,7 +651,7 @@ public class TestClientNoCluster extends
   private static ServerName [] makeServerNames(final int count) {
     ServerName [] sns = new ServerName[count];
     for (int i = 0; i < count; i++) {
-      sns[i] = new ServerName("" + i + ".example.org", 60010, i);
+      sns[i] = ServerName.valueOf("" + i + ".example.org", 60010, i);
     }
     return sns;
   }

Modified: hbase/branches/0.96/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RollingBatchRestartRsAction.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RollingBatchRestartRsAction.java?rev=1543923&r1=1543922&r2=1543923&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RollingBatchRestartRsAction.java (original)
+++ hbase/branches/0.96/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RollingBatchRestartRsAction.java Wed Nov 20 20:06:16 2013
@@ -99,7 +99,7 @@ public class RollingBatchRestartRsAction
         final int count = 4;
         List<ServerName> serverNames = new ArrayList<ServerName>(count);
         for (int i = 0; i < 4; i++) {
-          serverNames.add(new ServerName(i + ".example.org", i, i));
+          serverNames.add(ServerName.valueOf(i + ".example.org", i, i));
         }
         return serverNames.toArray(new ServerName [] {});
       }

Modified: hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java?rev=1543923&r1=1543922&r2=1543923&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java (original)
+++ hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java Wed Nov 20 20:06:16 2013
@@ -106,7 +106,7 @@ import com.google.common.collect.LinkedH
 public class AssignmentManager extends ZooKeeperListener {
   private static final Log LOG = LogFactory.getLog(AssignmentManager.class);
 
-  public static final ServerName HBCK_CODE_SERVERNAME = new ServerName(HConstants.HBCK_CODE_NAME,
+  public static final ServerName HBCK_CODE_SERVERNAME = ServerName.valueOf(HConstants.HBCK_CODE_NAME,
       -1, -1L);
 
   public static final String ASSIGNMENT_TIMEOUT = "hbase.master.assignment.timeoutmonitor.timeout";

Modified: hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java?rev=1543923&r1=1543922&r2=1543923&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java (original)
+++ hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java Wed Nov 20 20:06:16 2013
@@ -441,7 +441,7 @@ MasterServices, Server {
     // Set our address.
     this.isa = this.rpcServer.getListenerAddress();
     // We don't want to pass isa's hostname here since it could be 0.0.0.0
-    this.serverName = new ServerName(hostname, this.isa.getPort(), System.currentTimeMillis());
+    this.serverName = ServerName.valueOf(hostname, this.isa.getPort(), System.currentTimeMillis());
     this.rsFatals = new MemoryBoundedLogMessageBuffer(
       conf.getLong("hbase.master.buffer.for.rs.fatals", 1*1024*1024));
 
@@ -1700,7 +1700,7 @@ MasterServices, Server {
         regionState.getServerName());
       dest = balancer.randomAssignment(hri, destServers);
     } else {
-      dest = new ServerName(Bytes.toString(destServerName));
+      dest = ServerName.valueOf(Bytes.toString(destServerName));
       if (dest.equals(regionState.getServerName())) {
         LOG.debug("Skipping move of region " + hri.getRegionNameAsString()
           + " because region already assigned to the same server " + dest + ".");

Modified: hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java?rev=1543923&r1=1543922&r2=1543923&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java (original)
+++ hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java Wed Nov 20 20:06:16 2013
@@ -54,7 +54,6 @@ import org.apache.hadoop.hbase.master.ba
 import org.apache.hadoop.hbase.master.balancer.FavoredNodesPlan;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.RequestConverter;
-import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionRequest;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService.BlockingInterface;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse;
@@ -401,15 +400,15 @@ public class RegionPlacementMaintainer {
           List<ServerName> favoredServers =
             new ArrayList<ServerName>(FavoredNodeAssignmentHelper.FAVORED_NODES_NUM);
           ServerName s = servers.get(primaryAssignment[i] / slotsPerServer);
-          favoredServers.add(new ServerName(s.getHostname(), s.getPort(),
+          favoredServers.add(ServerName.valueOf(s.getHostname(), s.getPort(),
               ServerName.NON_STARTCODE));
 
           s = servers.get(secondaryAssignment[i] / slotsPerServer);
-          favoredServers.add(new ServerName(s.getHostname(), s.getPort(),
+          favoredServers.add(ServerName.valueOf(s.getHostname(), s.getPort(),
               ServerName.NON_STARTCODE));
 
           s = servers.get(tertiaryAssignment[i] / slotsPerServer);
-          favoredServers.add(new ServerName(s.getHostname(), s.getPort(),
+          favoredServers.add(ServerName.valueOf(s.getHostname(), s.getPort(),
               ServerName.NON_STARTCODE));
           // Update the assignment plan
           plan.updateAssignmentPlan(regions.get(i), favoredServers);
@@ -434,17 +433,17 @@ public class RegionPlacementMaintainer {
             new ArrayList<ServerName>(FavoredNodeAssignmentHelper.FAVORED_NODES_NUM);
           HRegionInfo currentRegion = regions.get(i);
           ServerName s = primaryRSMap.get(currentRegion);
-          favoredServers.add(new ServerName(s.getHostname(), s.getPort(),
+          favoredServers.add(ServerName.valueOf(s.getHostname(), s.getPort(),
               ServerName.NON_STARTCODE));
 
           ServerName[] secondaryAndTertiary =
               secondaryAndTertiaryMap.get(currentRegion);
           s = secondaryAndTertiary[0];
-          favoredServers.add(new ServerName(s.getHostname(), s.getPort(),
+          favoredServers.add(ServerName.valueOf(s.getHostname(), s.getPort(),
               ServerName.NON_STARTCODE));
 
           s = secondaryAndTertiary[1];
-          favoredServers.add(new ServerName(s.getHostname(), s.getPort(),
+          favoredServers.add(ServerName.valueOf(s.getHostname(), s.getPort(),
               ServerName.NON_STARTCODE));
           // Update the assignment plan
           plan.updateAssignmentPlan(regions.get(i), favoredServers);
@@ -943,7 +942,7 @@ public class RegionPlacementMaintainer {
 
     List<ServerName> serverList = new ArrayList<ServerName>();
     for (String hostNameAndPort : favoredNodesArray) {
-      serverList.add(new ServerName(hostNameAndPort, ServerName.NON_STARTCODE));
+      serverList.add(ServerName.valueOf(hostNameAndPort, ServerName.NON_STARTCODE));
     }
     return serverList;
   }

Modified: hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java?rev=1543923&r1=1543922&r2=1543923&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java (original)
+++ hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java Wed Nov 20 20:06:16 2013
@@ -211,7 +211,7 @@ public class ServerManager {
     // is, reject the server and trigger its expiration. The next time it comes
     // in, it should have been removed from serverAddressToServerInfo and queued
     // for processing by ProcessServerShutdown.
-    ServerName sn = new ServerName(ia.getHostName(), port, serverStartcode);
+    ServerName sn = ServerName.valueOf(ia.getHostName(), port, serverStartcode);
     checkClockSkew(sn, serverCurrentTime);
     checkIsDead(sn, "STARTUP");
     if (!checkAlreadySameHostPortAndRecordNewServer(

Modified: hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredNodeLoadBalancer.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredNodeLoadBalancer.java?rev=1543923&r1=1543922&r2=1543923&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredNodeLoadBalancer.java (original)
+++ hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredNodeLoadBalancer.java Wed Nov 20 20:06:16 2013
@@ -88,14 +88,14 @@ public class FavoredNodeLoadBalancer ext
         new HashMap<ServerName, ServerName>();
     ServerManager serverMgr = super.services.getServerManager();
     for (ServerName sn: serverMgr.getOnlineServersList()) {
-      ServerName s = new ServerName(sn.getHostname(), sn.getPort(), ServerName.NON_STARTCODE);
+      ServerName s = ServerName.valueOf(sn.getHostname(), sn.getPort(), ServerName.NON_STARTCODE);
       serverNameToServerNameWithoutCode.put(sn, s);
       serverNameWithoutCodeToServerName.put(s, sn);
     }
     for (Map.Entry<ServerName, List<HRegionInfo>> entry : clusterState.entrySet()) {
       ServerName currentServer = entry.getKey();
       //get a server without the startcode for the currentServer
-      ServerName currentServerWithoutStartCode = new ServerName(currentServer.getHostname(),
+      ServerName currentServerWithoutStartCode = ServerName.valueOf(currentServer.getHostname(),
           currentServer.getPort(), ServerName.NON_STARTCODE);
       List<HRegionInfo> list = entry.getValue();
       for (HRegionInfo region : list) {
@@ -332,13 +332,13 @@ public class FavoredNodeLoadBalancer ext
       // We don't care about the startcode; but only the hostname really
       List<ServerName> favoredNodesForRegion = new ArrayList<ServerName>(3);
       ServerName sn = primaryRSMap.get(region);
-      favoredNodesForRegion.add(new ServerName(sn.getHostname(), sn.getPort(),
+      favoredNodesForRegion.add(ServerName.valueOf(sn.getHostname(), sn.getPort(),
           ServerName.NON_STARTCODE));
       ServerName[] secondaryAndTertiaryNodes = secondaryAndTertiaryRSMap.get(region);
       if (secondaryAndTertiaryNodes != null) {
-        favoredNodesForRegion.add(new ServerName(secondaryAndTertiaryNodes[0].getHostname(),
+        favoredNodesForRegion.add(ServerName.valueOf(secondaryAndTertiaryNodes[0].getHostname(),
             secondaryAndTertiaryNodes[0].getPort(), ServerName.NON_STARTCODE));
-        favoredNodesForRegion.add(new ServerName(secondaryAndTertiaryNodes[1].getHostname(),
+        favoredNodesForRegion.add(ServerName.valueOf(secondaryAndTertiaryNodes[1].getHostname(),
             secondaryAndTertiaryNodes[1].getPort(), ServerName.NON_STARTCODE));
       }
       globalFavoredNodesAssignmentPlan.updateFavoredNodesMap(region, favoredNodesForRegion);

Modified: hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/migration/NamespaceUpgrade.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/migration/NamespaceUpgrade.java?rev=1543923&r1=1543922&r2=1543923&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/migration/NamespaceUpgrade.java (original)
+++ hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/migration/NamespaceUpgrade.java Wed Nov 20 20:06:16 2013
@@ -378,7 +378,7 @@ public class NamespaceUpgrade implements
     }
 
 
-    ServerName fakeServer = new ServerName("nsupgrade",96,123);
+    ServerName fakeServer = ServerName.valueOf("nsupgrade", 96, 123);
     String metaLogName = HLogUtil.getHLogDirectoryName(fakeServer.toString());
     HLog metaHLog = HLogFactory.createMetaHLog(fs, rootDir,
         metaLogName, conf, null,

Modified: hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java?rev=1543923&r1=1543922&r2=1543923&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java (original)
+++ hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java Wed Nov 20 20:06:16 2013
@@ -714,7 +714,7 @@ public class HRegionServer implements Cl
       this.abort("Failed to reach zk cluster when creating snapshot handler.");
     }
     this.tableLockManager = TableLockManager.createTableLockManager(conf, zooKeeper,
-        new ServerName(isa.getHostName(), isa.getPort(), startcode));
+        ServerName.valueOf(isa.getHostName(), isa.getPort(), startcode));
 
     // register watcher for recovering regions
     if(this.distributedLogReplay) {
@@ -1152,8 +1152,8 @@ public class HRegionServer implements Cl
         // The hostname the master sees us as.
         if (key.equals(HConstants.KEY_FOR_HOSTNAME_SEEN_BY_MASTER)) {
           String hostnameFromMasterPOV = e.getValue();
-          this.serverNameFromMasterPOV = new ServerName(hostnameFromMasterPOV,
-            this.isa.getPort(), this.startcode);
+          this.serverNameFromMasterPOV = ServerName.valueOf(hostnameFromMasterPOV,
+              this.isa.getPort(), this.startcode);
           if (!hostnameFromMasterPOV.equals(this.isa.getHostName())) {
             LOG.info("Master passed us a different hostname to use; was=" +
               this.isa.getHostName() + ", but now=" + hostnameFromMasterPOV);
@@ -2173,7 +2173,7 @@ public class HRegionServer implements Cl
   public ServerName getServerName() {
     // Our servername could change after we talk to the master.
     return this.serverNameFromMasterPOV == null?
-      new ServerName(this.isa.getHostName(), this.isa.getPort(), this.startcode):
+        ServerName.valueOf(this.isa.getHostName(), this.isa.getPort(), this.startcode) :
         this.serverNameFromMasterPOV;
   }
 

Modified: hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/DrainingServerTracker.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/DrainingServerTracker.java?rev=1543923&r1=1543922&r2=1543923&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/DrainingServerTracker.java (original)
+++ hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/DrainingServerTracker.java Wed Nov 20 20:06:16 2013
@@ -77,7 +77,7 @@ public class DrainingServerTracker exten
     synchronized(this.drainingServers) {
       this.drainingServers.clear();
       for (String n: servers) {
-        final ServerName sn = new ServerName(ZKUtil.getNodeName(n));
+        final ServerName sn = ServerName.valueOf(ZKUtil.getNodeName(n));
         this.drainingServers.add(sn);
         this.serverManager.addServerToDrainList(sn);
         LOG.info("Draining RS node created, adding to list [" +
@@ -97,7 +97,7 @@ public class DrainingServerTracker exten
   @Override
   public void nodeDeleted(final String path) {
     if(path.startsWith(watcher.drainingZNode)) {
-      final ServerName sn = new ServerName(ZKUtil.getNodeName(path));
+      final ServerName sn = ServerName.valueOf(ZKUtil.getNodeName(path));
       LOG.info("Draining RS node deleted, removing from list [" +
           sn + "]");
       remove(sn);

Modified: hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/TestDrainingServer.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/TestDrainingServer.java?rev=1543923&r1=1543922&r2=1543923&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/TestDrainingServer.java (original)
+++ hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/TestDrainingServer.java Wed Nov 20 20:06:16 2013
@@ -89,8 +89,8 @@ public class TestDrainingServer {
     final HMaster master = Mockito.mock(HMaster.class);
     final Server server = Mockito.mock(Server.class);
     final ServerManager serverManager = Mockito.mock(ServerManager.class);
-    final ServerName SERVERNAME_A = new ServerName("mockserver_a.org", 1000, 8000);
-    final ServerName SERVERNAME_B = new ServerName("mockserver_b.org", 1001, 8000);
+    final ServerName SERVERNAME_A = ServerName.valueOf("mockserver_a.org", 1000, 8000);
+    final ServerName SERVERNAME_B = ServerName.valueOf("mockserver_b.org", 1001, 8000);
     LoadBalancer balancer = LoadBalancerFactory.getLoadBalancer(conf);
     CatalogTracker catalogTracker = Mockito.mock(CatalogTracker.class);
     final HRegionInfo REGIONINFO = new HRegionInfo(TableName.valueOf("table_test"),
@@ -105,7 +105,7 @@ public class TestDrainingServer {
     onlineServers.put(SERVERNAME_B, ServerLoad.EMPTY_SERVERLOAD);
 
     Mockito.when(server.getConfiguration()).thenReturn(conf);
-    Mockito.when(server.getServerName()).thenReturn(new ServerName("masterMock,1,1"));
+    Mockito.when(server.getServerName()).thenReturn(ServerName.valueOf("masterMock,1,1"));
     Mockito.when(server.getZooKeeper()).thenReturn(zkWatcher);
 
     Mockito.when(serverManager.getOnlineServers()).thenReturn(onlineServers);
@@ -163,11 +163,11 @@ public class TestDrainingServer {
     final HMaster master = Mockito.mock(HMaster.class);
     final Server server = Mockito.mock(Server.class);
     final ServerManager serverManager = Mockito.mock(ServerManager.class);
-    final ServerName SERVERNAME_A = new ServerName("mockserverbulk_a.org", 1000, 8000);
-    final ServerName SERVERNAME_B = new ServerName("mockserverbulk_b.org", 1001, 8000);
-    final ServerName SERVERNAME_C = new ServerName("mockserverbulk_c.org", 1002, 8000);
-    final ServerName SERVERNAME_D = new ServerName("mockserverbulk_d.org", 1003, 8000);
-    final ServerName SERVERNAME_E = new ServerName("mockserverbulk_e.org", 1004, 8000);
+    final ServerName SERVERNAME_A = ServerName.valueOf("mockserverbulk_a.org", 1000, 8000);
+    final ServerName SERVERNAME_B = ServerName.valueOf("mockserverbulk_b.org", 1001, 8000);
+    final ServerName SERVERNAME_C = ServerName.valueOf("mockserverbulk_c.org", 1002, 8000);
+    final ServerName SERVERNAME_D = ServerName.valueOf("mockserverbulk_d.org", 1003, 8000);
+    final ServerName SERVERNAME_E = ServerName.valueOf("mockserverbulk_e.org", 1004, 8000);
     final Map<HRegionInfo, ServerName> bulk = new HashMap<HRegionInfo, ServerName>();
 
     Set<ServerName> bunchServersAssigned = new HashSet<ServerName>();
@@ -202,7 +202,7 @@ public class TestDrainingServer {
         "zkWatcher-BulkAssignTest", abortable, true);
 
     Mockito.when(server.getConfiguration()).thenReturn(conf);
-    Mockito.when(server.getServerName()).thenReturn(new ServerName("masterMock,1,1"));
+    Mockito.when(server.getServerName()).thenReturn(ServerName.valueOf("masterMock,1,1"));
     Mockito.when(server.getZooKeeper()).thenReturn(zkWatcher);
 
     Mockito.when(serverManager.getOnlineServers()).thenReturn(onlineServers);

Modified: hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHRegionLocation.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHRegionLocation.java?rev=1543923&r1=1543922&r2=1543923&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHRegionLocation.java (original)
+++ hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHRegionLocation.java Wed Nov 20 20:06:16 2013
@@ -35,7 +35,7 @@ public class TestHRegionLocation {
    */
   @Test
   public void testHashAndEqualsCode() {
-    ServerName hsa1 = new ServerName("localhost", 1234, -1L);
+    ServerName hsa1 = ServerName.valueOf("localhost", 1234, -1L);
     HRegionLocation hrl1 = new HRegionLocation(HRegionInfo.FIRST_META_REGIONINFO, hsa1);
     HRegionLocation hrl2 = new HRegionLocation(HRegionInfo.FIRST_META_REGIONINFO, hsa1);
     assertEquals(hrl1.hashCode(), hrl2.hashCode());
@@ -45,7 +45,7 @@ public class TestHRegionLocation {
     // They are equal because they have same location even though they are
     // carrying different regions or timestamp.
     assertTrue(hrl1.equals(hrl3));
-    ServerName hsa2 = new ServerName("localhost", 12345, -1L);
+    ServerName hsa2 = ServerName.valueOf("localhost", 12345, -1L);
     HRegionLocation hrl4 = new HRegionLocation(HRegionInfo.FIRST_META_REGIONINFO, hsa2);
     // These have same HRI but different locations so should be different.
     assertFalse(hrl3.equals(hrl4));
@@ -56,17 +56,17 @@ public class TestHRegionLocation {
 
   @Test
   public void testToString() {
-    ServerName hsa1 = new ServerName("localhost", 1234, -1L);
+    ServerName hsa1 = ServerName.valueOf("localhost", 1234, -1L);
     HRegionLocation hrl1 = new HRegionLocation(HRegionInfo.FIRST_META_REGIONINFO, hsa1);
     System.out.println(hrl1.toString());
   }
 
   @Test
   public void testCompareTo() {
-    ServerName hsa1 = new ServerName("localhost", 1234, -1L);
+    ServerName hsa1 = ServerName.valueOf("localhost", 1234, -1L);
     HRegionLocation hsl1 =
       new HRegionLocation(HRegionInfo.FIRST_META_REGIONINFO, hsa1);
-    ServerName hsa2 = new ServerName("localhost", 1235, -1L);
+    ServerName hsa2 = ServerName.valueOf("localhost", 1235, -1L);
     HRegionLocation hsl2 =
       new HRegionLocation(HRegionInfo.FIRST_META_REGIONINFO, hsa2);
     assertTrue(hsl1.compareTo(hsl1) == 0);

Modified: hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/TestSerialization.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/TestSerialization.java?rev=1543923&r1=1543922&r2=1543923&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/TestSerialization.java (original)
+++ hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/TestSerialization.java Wed Nov 20 20:06:16 2013
@@ -77,7 +77,7 @@ public class TestSerialization {
   
   @Test
   public void testSplitLogTask() throws DeserializationException {
-    SplitLogTask slt = new SplitLogTask.Unassigned(new ServerName("mgr,1,1"));
+    SplitLogTask slt = new SplitLogTask.Unassigned(ServerName.valueOf("mgr,1,1"));
     byte [] bytes = slt.toByteArray();
     SplitLogTask sltDeserialized = SplitLogTask.parseFrom(bytes);
     assertTrue(slt.equals(sltDeserialized));

Modified: hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerName.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerName.java?rev=1543923&r1=1543922&r2=1543923&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerName.java (original)
+++ hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerName.java Wed Nov 20 20:06:16 2013
@@ -41,17 +41,17 @@ public class TestServerName {
     assertEquals("x", ServerName.getHostNameMinusDomain("x"));
     assertEquals("x", ServerName.getHostNameMinusDomain("x.y.z"));
     assertEquals("asf000", ServerName.getHostNameMinusDomain("asf000.sp2.ygridcore.net"));
-    ServerName sn = new ServerName("asf000.sp2.ygridcore.net", 1, 1);
+    ServerName sn = ServerName.valueOf("asf000.sp2.ygridcore.net", 1, 1);
     assertEquals("asf000.sp2.ygridcore.net,1,1", sn.toString());
   }
 
   @Test
   public void testShortString() {
-    ServerName sn = new ServerName("asf000.sp2.ygridcore.net", 1, 1);
+    ServerName sn = ServerName.valueOf("asf000.sp2.ygridcore.net", 1, 1);
     assertEquals("asf000:1", sn.toShortString());
-    sn = new ServerName("2607:f0d0:1002:0051:0000:0000:0000:0004", 1, 1);
+    sn = ServerName.valueOf("2607:f0d0:1002:0051:0000:0000:0000:0004", 1, 1);
     assertEquals("2607:f0d0:1002:0051:0000:0000:0000:0004:1", sn.toShortString());
-    sn = new ServerName("1.1.1.1", 1, 1);
+    sn = ServerName.valueOf("1.1.1.1", 1, 1);
     assertEquals("1.1.1.1:1", sn.toShortString());
   }
 
@@ -68,7 +68,7 @@ public class TestServerName {
 
   @Test public void testParseOfBytes() {
     final String snStr = "www.example.org,1234,5678";
-    ServerName sn = new ServerName(snStr);
+    ServerName sn = ServerName.valueOf(snStr);
     byte [] versionedBytes = sn.getVersionedBytes();
     assertEquals(sn.toString(), ServerName.parseVersionedServerName(versionedBytes).toString());
     final String hostnamePortStr = sn.getHostAndPort();
@@ -81,9 +81,9 @@ public class TestServerName {
 
   @Test
   public void testServerName() {
-    ServerName sn = new ServerName("www.example.org", 1234, 5678);
-    ServerName sn2 = new ServerName("www.example.org", 1234, 5678);
-    ServerName sn3 = new ServerName("www.example.org", 1234, 56789);
+    ServerName sn = ServerName.valueOf("www.example.org", 1234, 5678);
+    ServerName sn2 = ServerName.valueOf("www.example.org", 1234, 5678);
+    ServerName sn3 = ServerName.valueOf("www.example.org", 1234, 56789);
     assertTrue(sn.equals(sn2));
     assertFalse(sn.equals(sn3));
     assertEquals(sn.hashCode(), sn2.hashCode());
@@ -99,7 +99,7 @@ public class TestServerName {
 
   @Test
   public void getServerStartcodeFromServerName() {
-    ServerName sn = new ServerName("www.example.org", 1234, 5678);
+    ServerName sn = ServerName.valueOf("www.example.org", 1234, 5678);
     assertEquals(5678,
       ServerName.getServerStartcodeFromServerName(sn.toString()));
     assertNotSame(5677,

Modified: hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestCatalogTracker.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestCatalogTracker.java?rev=1543923&r1=1543922&r2=1543923&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestCatalogTracker.java (original)
+++ hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestCatalogTracker.java Wed Nov 20 20:06:16 2013
@@ -72,7 +72,7 @@ public class TestCatalogTracker {
   private static final Log LOG = LogFactory.getLog(TestCatalogTracker.class);
   private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
   private static final ServerName SN =
-    new ServerName("example.org", 1234, System.currentTimeMillis());
+      ServerName.valueOf("example.org", 1234, System.currentTimeMillis());
   private ZooKeeperWatcher watcher;
   private Abortable abortable;
 
@@ -137,7 +137,7 @@ public class TestCatalogTracker {
     constructAndStartCatalogTracker(connection);
 
     MetaRegionTracker.setMetaLocation(this.watcher,
-        new ServerName("example.com", 1234, System.currentTimeMillis()));
+        ServerName.valueOf("example.com", 1234, System.currentTimeMillis()));
   }
 
   /**
@@ -251,7 +251,7 @@ public class TestCatalogTracker {
     final CatalogTracker ct = constructAndStartCatalogTracker(connection);
 
     MetaRegionTracker.setMetaLocation(this.watcher,
-        new ServerName("example.com", 1234, System.currentTimeMillis()));
+        ServerName.valueOf("example.com", 1234, System.currentTimeMillis()));
     Assert.assertFalse(ct.verifyMetaRegionLocation(100));
   }
 

Modified: hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaReaderEditorNoCluster.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaReaderEditorNoCluster.java?rev=1543923&r1=1543922&r2=1543923&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaReaderEditorNoCluster.java (original)
+++ hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaReaderEditorNoCluster.java Wed Nov 20 20:06:16 2013
@@ -137,7 +137,7 @@ public class TestMetaReaderEditorNoClust
     ZooKeeperWatcher zkw = new ZooKeeperWatcher(UTIL.getConfiguration(),
       this.getClass().getSimpleName(), ABORTABLE, true);
     // This is a servername we use in a few places below.
-    ServerName sn = new ServerName("example.com", 1234, System.currentTimeMillis());
+    ServerName sn = ServerName.valueOf("example.com", 1234, System.currentTimeMillis());
 
     HConnection connection;
     CatalogTracker ct = null;

Modified: hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java?rev=1543923&r1=1543922&r2=1543923&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java (original)
+++ hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java Wed Nov 20 20:06:16 2013
@@ -361,8 +361,8 @@ public class TestHCM {
 
     final int nextPort = conn.getCachedLocation(TABLE_NAME, ROW).getPort() + 1;
     HRegionLocation loc = conn.getCachedLocation(TABLE_NAME, ROW);
-    conn.updateCachedLocation(loc.getRegionInfo(), loc, new ServerName("127.0.0.1", nextPort,
-      HConstants.LATEST_TIMESTAMP), HConstants.LATEST_TIMESTAMP);
+    conn.updateCachedLocation(loc.getRegionInfo(), loc, ServerName.valueOf("127.0.0.1", nextPort,
+        HConstants.LATEST_TIMESTAMP), HConstants.LATEST_TIMESTAMP);
     Assert.assertEquals(conn.getCachedLocation(TABLE_NAME, ROW).getPort(), nextPort);
 
     conn.forceDeleteCachedLocation(TABLE_NAME, ROW.clone());
@@ -553,34 +553,34 @@ public class TestHCM {
     HRegionLocation location = conn.getCachedLocation(TABLE_NAME2, ROW);
     assertNotNull(location);
 
-    HRegionLocation anySource = new HRegionLocation(location.getRegionInfo(), new ServerName(
+    HRegionLocation anySource = new HRegionLocation(location.getRegionInfo(), ServerName.valueOf(
         location.getHostname(), location.getPort() - 1, 0L));
 
     // Same server as already in cache reporting - overwrites any value despite seqNum.
     int nextPort = location.getPort() + 1;
     conn.updateCachedLocation(location.getRegionInfo(), location,
-        new ServerName("127.0.0.1", nextPort, 0), location.getSeqNum() - 1);
+        ServerName.valueOf("127.0.0.1", nextPort, 0), location.getSeqNum() - 1);
     location = conn.getCachedLocation(TABLE_NAME2, ROW);
     Assert.assertEquals(nextPort, location.getPort());
 
     // No source specified - same.
     nextPort = location.getPort() + 1;
     conn.updateCachedLocation(location.getRegionInfo(), location,
-        new ServerName("127.0.0.1", nextPort, 0), location.getSeqNum() - 1);
+        ServerName.valueOf("127.0.0.1", nextPort, 0), location.getSeqNum() - 1);
     location = conn.getCachedLocation(TABLE_NAME2, ROW);
     Assert.assertEquals(nextPort, location.getPort());
 
     // Higher seqNum - overwrites lower seqNum.
     nextPort = location.getPort() + 1;
     conn.updateCachedLocation(location.getRegionInfo(), anySource,
-        new ServerName("127.0.0.1", nextPort, 0), location.getSeqNum() + 1);
+        ServerName.valueOf("127.0.0.1", nextPort, 0), location.getSeqNum() + 1);
     location = conn.getCachedLocation(TABLE_NAME2, ROW);
     Assert.assertEquals(nextPort, location.getPort());
 
     // Lower seqNum - does not overwrite higher seqNum.
     nextPort = location.getPort() + 1;
     conn.updateCachedLocation(location.getRegionInfo(), anySource,
-        new ServerName("127.0.0.1", nextPort, 0), location.getSeqNum() - 1);
+        ServerName.valueOf("127.0.0.1", nextPort, 0), location.getSeqNum() - 1);
     location = conn.getCachedLocation(TABLE_NAME2, ROW);
     Assert.assertEquals(nextPort - 1, location.getPort());
   }
@@ -868,8 +868,8 @@ public class TestHCM {
     // TODO: This test would seem to presume hardcoded RETRY_BACKOFF which it should not.
     final long ANY_PAUSE = 100;
     HRegionInfo ri = new HRegionInfo(TABLE_NAME);
-    HRegionLocation location = new HRegionLocation(ri, new ServerName("127.0.0.1", 1, 0));
-    HRegionLocation diffLocation = new HRegionLocation(ri, new ServerName("127.0.0.1", 2, 0));
+    HRegionLocation location = new HRegionLocation(ri, ServerName.valueOf("127.0.0.1", 1, 0));
+    HRegionLocation diffLocation = new HRegionLocation(ri, ServerName.valueOf("127.0.0.1", 2, 0));
 
     ManualEnvironmentEdge timeMachine = new ManualEnvironmentEdge();
     EnvironmentEdgeManager.injectEdge(timeMachine);

Modified: hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaScanner.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaScanner.java?rev=1543923&r1=1543922&r2=1543923&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaScanner.java (original)
+++ hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaScanner.java Wed Nov 20 20:06:16 2013
@@ -169,7 +169,7 @@ public class TestMetaScanner {
               Bytes.toBytes(midKey),
               end);
 
-            MetaEditor.splitRegion(catalogTracker, parent, splita, splitb, new ServerName("fooserver", 1, 0));
+            MetaEditor.splitRegion(catalogTracker, parent, splita, splitb, ServerName.valueOf("fooserver", 1, 0));
 
             Threads.sleep(random.nextInt(200));
           } catch (Throwable e) {

Modified: hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestDelayedRpc.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestDelayedRpc.java?rev=1543923&r1=1543922&r2=1543923&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestDelayedRpc.java (original)
+++ hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestDelayedRpc.java Wed Nov 20 20:06:16 2013
@@ -89,7 +89,7 @@ public class TestDelayedRpc {
     RpcClient rpcClient = new RpcClient(conf, HConstants.DEFAULT_CLUSTER_ID.toString());
     try {
       BlockingRpcChannel channel = rpcClient.createBlockingRpcChannel(
-          new ServerName(rpcServer.getListenerAddress().getHostName(),
+          ServerName.valueOf(rpcServer.getListenerAddress().getHostName(),
               rpcServer.getListenerAddress().getPort(), System.currentTimeMillis()),
           User.getCurrent(), RPC_CLIENT_TIMEOUT);
       TestDelayedRpcProtos.TestDelayedService.BlockingInterface stub =
@@ -168,7 +168,7 @@ public class TestDelayedRpc {
     RpcClient rpcClient = new RpcClient(conf, HConstants.DEFAULT_CLUSTER_ID.toString());
     try {
       BlockingRpcChannel channel = rpcClient.createBlockingRpcChannel(
-          new ServerName(rpcServer.getListenerAddress().getHostName(),
+          ServerName.valueOf(rpcServer.getListenerAddress().getHostName(),
               rpcServer.getListenerAddress().getPort(), System.currentTimeMillis()),
           User.getCurrent(), RPC_CLIENT_TIMEOUT);
       TestDelayedRpcProtos.TestDelayedService.BlockingInterface stub =
@@ -288,8 +288,8 @@ public class TestDelayedRpc {
     RpcClient rpcClient = new RpcClient(conf, HConstants.DEFAULT_CLUSTER_ID.toString());
     try {
       BlockingRpcChannel channel = rpcClient.createBlockingRpcChannel(
-        new ServerName(rpcServer.getListenerAddress().getHostName(),
-          rpcServer.getListenerAddress().getPort(), System.currentTimeMillis()),
+          ServerName.valueOf(rpcServer.getListenerAddress().getHostName(),
+              rpcServer.getListenerAddress().getPort(), System.currentTimeMillis()),
         User.getCurrent(), 1000);
       TestDelayedRpcProtos.TestDelayedService.BlockingInterface stub =
         TestDelayedRpcProtos.TestDelayedService.newBlockingStub(channel);

Modified: hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestProtoBufRpc.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestProtoBufRpc.java?rev=1543923&r1=1543922&r2=1543923&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestProtoBufRpc.java (original)
+++ hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestProtoBufRpc.java Wed Nov 20 20:06:16 2013
@@ -113,7 +113,7 @@ public class TestProtoBufRpc {
     RpcClient rpcClient = new RpcClient(conf, HConstants.CLUSTER_ID_DEFAULT);
     try {
       BlockingRpcChannel channel = rpcClient.createBlockingRpcChannel(
-        new ServerName(this.isa.getHostName(), this.isa.getPort(), System.currentTimeMillis()),
+          ServerName.valueOf(this.isa.getHostName(), this.isa.getPort(), System.currentTimeMillis()),
         User.getCurrent(), 0);
       TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface stub =
         TestRpcServiceProtos.TestProtobufRpcProto.newBlockingStub(channel);

Modified: hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesSplitRecovery.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesSplitRecovery.java?rev=1543923&r1=1543922&r2=1543923&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesSplitRecovery.java (original)
+++ hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesSplitRecovery.java Wed Nov 20 20:06:16 2013
@@ -274,7 +274,7 @@ public class TestLoadIncrementalHFilesSp
     Mockito.doNothing().when(c).close();
     // Make it so we return a particular location when asked.
     final HRegionLocation loc = new HRegionLocation(HRegionInfo.FIRST_META_REGIONINFO,
-        new ServerName("example.org", 1234, 0));
+        ServerName.valueOf("example.org", 1234, 0));
     Mockito.when(c.getRegionLocation((TableName) Mockito.any(),
         (byte[]) Mockito.any(), Mockito.anyBoolean())).
       thenReturn(loc);

Modified: hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java?rev=1543923&r1=1543922&r2=1543923&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java (original)
+++ hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java Wed Nov 20 20:06:16 2013
@@ -73,7 +73,7 @@ public class TestActiveMasterManager {
     } catch(KeeperException.NoNodeException nne) {}
 
     // Create the master node with a dummy address
-    ServerName master = new ServerName("localhost", 1, System.currentTimeMillis());
+    ServerName master = ServerName.valueOf("localhost", 1, System.currentTimeMillis());
     // Should not have a master yet
     DummyMaster dummyMaster = new DummyMaster(zk,master);
     ClusterStatusTracker clusterStatusTracker =
@@ -116,9 +116,9 @@ public class TestActiveMasterManager {
 
     // Create the master node with a dummy address
     ServerName firstMasterAddress =
-      new ServerName("localhost", 1, System.currentTimeMillis());
+        ServerName.valueOf("localhost", 1, System.currentTimeMillis());
     ServerName secondMasterAddress =
-      new ServerName("localhost", 2, System.currentTimeMillis());
+        ServerName.valueOf("localhost", 2, System.currentTimeMillis());
 
     // Should not have a master yet
     DummyMaster ms1 = new DummyMaster(zk,firstMasterAddress);

Modified: hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManager.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManager.java?rev=1543923&r1=1543922&r2=1543923&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManager.java (original)
+++ hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManager.java Wed Nov 20 20:06:16 2013
@@ -100,9 +100,9 @@ import com.google.protobuf.ServiceExcept
 public class TestAssignmentManager {
   private static final HBaseTestingUtility HTU = new HBaseTestingUtility();
   private static final ServerName SERVERNAME_A =
-    new ServerName("example.org", 1234, 5678);
+      ServerName.valueOf("example.org", 1234, 5678);
   private static final ServerName SERVERNAME_B =
-    new ServerName("example.org", 0, 5678);
+      ServerName.valueOf("example.org", 0, 5678);
   private static final HRegionInfo REGIONINFO =
     new HRegionInfo(TableName.valueOf("t"),
       HConstants.EMPTY_START_ROW, HConstants.EMPTY_START_ROW);
@@ -135,7 +135,7 @@ public class TestAssignmentManager {
     // If abort is called, be sure to fail the test (don't just swallow it
     // silently as is mockito default).
     this.server = Mockito.mock(Server.class);
-    Mockito.when(server.getServerName()).thenReturn(new ServerName("master,1,1"));
+    Mockito.when(server.getServerName()).thenReturn(ServerName.valueOf("master,1,1"));
     Mockito.when(server.getConfiguration()).thenReturn(HTU.getConfiguration());
     this.watcher =
       new ZooKeeperWatcher(HTU.getConfiguration(), "mockedServer", this.server, true);

Modified: hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java?rev=1543923&r1=1543922&r2=1543923&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java (original)
+++ hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java Wed Nov 20 20:06:16 2013
@@ -158,8 +158,8 @@ public class TestAssignmentManagerOnClus
       ServerName destServer = onlineServers.iterator().next();
 
       // Created faked dead server
-      deadServer = new ServerName(destServer.getHostname(),
-        destServer.getPort(), destServer.getStartcode() - 100L);
+      deadServer = ServerName.valueOf(destServer.getHostname(),
+          destServer.getPort(), destServer.getStartcode() - 100L);
       master.serverManager.recordNewServer(deadServer, ServerLoad.EMPTY_SERVERLOAD);
 
       AssignmentManager am = master.getAssignmentManager();

Modified: hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java?rev=1543923&r1=1543922&r2=1543923&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java (original)
+++ hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java Wed Nov 20 20:06:16 2013
@@ -111,7 +111,7 @@ public class TestCatalogJanitor {
       this.connection =
         HConnectionTestingUtility.getMockedConnectionAndDecorate(this.c,
           Mockito.mock(AdminProtos.AdminService.BlockingInterface.class), ri,
-          new ServerName("example.org,12345,6789"),
+            ServerName.valueOf("example.org,12345,6789"),
           HRegionInfo.FIRST_META_REGIONINFO);
       // Set hbase.rootdir into test dir.
       FileSystem fs = FileSystem.get(this.c);
@@ -136,7 +136,7 @@ public class TestCatalogJanitor {
 
     @Override
     public ServerName getServerName() {
-      return new ServerName("mockserver.example.org", 1234, -1L);
+      return ServerName.valueOf("mockserver.example.org", 1234, -1L);
     }
 
     @Override

Modified: hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClusterStatusPublisher.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClusterStatusPublisher.java?rev=1543923&r1=1543922&r2=1543923&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClusterStatusPublisher.java (original)
+++ hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClusterStatusPublisher.java Wed Nov 20 20:06:16 2013
@@ -63,7 +63,7 @@ public class TestClusterStatusPublisher 
         List<Pair<ServerName, Long>> res = new ArrayList<Pair<ServerName, Long>>();
         switch ((int) EnvironmentEdgeManager.currentTimeMillis()) {
           case 2:
-            res.add(new Pair<ServerName, Long>(new ServerName("hn", 10, 10), 1L));
+            res.add(new Pair<ServerName, Long>(ServerName.valueOf("hn", 10, 10), 1L));
             break;
           case 1000:
             break;
@@ -88,7 +88,7 @@ public class TestClusterStatusPublisher 
       protected List<Pair<ServerName, Long>> getDeadServers(long since) {
         List<Pair<ServerName, Long>> res = new ArrayList<Pair<ServerName, Long>>();
         for (int i = 0; i < 25; i++) {
-          res.add(new Pair<ServerName, Long>(new ServerName("hn" + i, 10, 10), 20L));
+          res.add(new Pair<ServerName, Long>(ServerName.valueOf("hn" + i, 10, 10), 20L));
         }
 
         return res;

Modified: hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDeadServer.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDeadServer.java?rev=1543923&r1=1543922&r2=1543923&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDeadServer.java (original)
+++ hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDeadServer.java Wed Nov 20 20:06:16 2013
@@ -34,10 +34,10 @@ import static org.junit.Assert.assertTru
 
 @Category(MediumTests.class)
 public class TestDeadServer {
-  final ServerName hostname123 = new ServerName("127.0.0.1", 123, 3L);
-  final ServerName hostname123_2 = new ServerName("127.0.0.1", 123, 4L);
-  final ServerName hostname1234 = new ServerName("127.0.0.2", 1234, 4L);
-  final ServerName hostname12345 = new ServerName("127.0.0.2", 12345, 4L);
+  final ServerName hostname123 = ServerName.valueOf("127.0.0.1", 123, 3L);
+  final ServerName hostname123_2 = ServerName.valueOf("127.0.0.1", 123, 4L);
+  final ServerName hostname1234 = ServerName.valueOf("127.0.0.2", 1234, 4L);
+  final ServerName hostname12345 = ServerName.valueOf("127.0.0.2", 12345, 4L);
 
   @Test public void testIsDead() {
     DeadServer ds = new DeadServer();
@@ -59,7 +59,7 @@ public class TestDeadServer {
     // Already dead =       127.0.0.1,9090,112321
     // Coming back alive =  127.0.0.1,9090,223341
 
-    final ServerName deadServer = new ServerName("127.0.0.1", 9090, 112321L);
+    final ServerName deadServer = ServerName.valueOf("127.0.0.1", 9090, 112321L);
     assertFalse(ds.cleanPreviousInstance(deadServer));
     ds.add(deadServer);
     assertTrue(ds.isDeadServer(deadServer));
@@ -68,7 +68,7 @@ public class TestDeadServer {
       Assert.assertNotNull(ds.getTimeOfDeath(eachDeadServer));
     }
     final ServerName deadServerHostComingAlive =
-      new ServerName("127.0.0.1", 9090, 223341L);
+        ServerName.valueOf("127.0.0.1", 9090, 223341L);
     assertTrue(ds.cleanPreviousInstance(deadServerHostComingAlive));
     assertFalse(ds.isDeadServer(deadServer));
     assertFalse(ds.cleanPreviousInstance(deadServerHostComingAlive));

Modified: hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystem.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystem.java?rev=1543923&r1=1543922&r2=1543923&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystem.java (original)
+++ hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystem.java Wed Nov 20 20:06:16 2013
@@ -26,13 +26,11 @@ import java.util.Set;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.MediumTests;
-import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.SplitLogTask;
 import org.apache.hadoop.hbase.util.FSUtils;
@@ -89,8 +87,8 @@ public class TestMasterFileSystem {
 
     String failedRegion = "failedRegoin1";
     String staleRegion = "staleRegion";
-    ServerName inRecoveryServerName = new ServerName("mgr,1,1");
-    ServerName previouselyFaildServerName = new ServerName("previous,1,1");
+    ServerName inRecoveryServerName = ServerName.valueOf("mgr,1,1");
+    ServerName previouselyFaildServerName = ServerName.valueOf("previous,1,1");
     String walPath = "/hbase/data/.logs/" + inRecoveryServerName.getServerName()
         + "-splitting/test";
     // Create a ZKW to use in the test

Modified: hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java?rev=1543923&r1=1543922&r2=1543923&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java (original)
+++ hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java Wed Nov 20 20:06:16 2013
@@ -144,9 +144,9 @@ public class TestMasterNoCluster {
     final long now = System.currentTimeMillis();
     // Names for our three servers.  Make the port numbers match hostname.
     // Will come in use down in the server when we need to figure how to respond.
-    final ServerName sn0 = new ServerName("0.example.org", 0, now);
-    final ServerName sn1 = new ServerName("1.example.org", 1, now);
-    final ServerName sn2 = new ServerName("2.example.org", 2, now);
+    final ServerName sn0 = ServerName.valueOf("0.example.org", 0, now);
+    final ServerName sn1 = ServerName.valueOf("1.example.org", 1, now);
+    final ServerName sn2 = ServerName.valueOf("2.example.org", 2, now);
     final ServerName [] sns = new ServerName [] {sn0, sn1, sn2};
     // Put up the mock servers
     final Configuration conf = TESTUTIL.getConfiguration();
@@ -256,7 +256,7 @@ public class TestMasterNoCluster {
 
     final long now = System.currentTimeMillis();
     // Name for our single mocked up regionserver.
-    final ServerName sn = new ServerName("0.example.org", 0, now);
+    final ServerName sn = ServerName.valueOf("0.example.org", 0, now);
     // Here is our mocked up regionserver.  Create it now.  Need it setting up
     // master next.
     final MockRegionServer rs0 = new MockRegionServer(conf, sn);
@@ -357,8 +357,8 @@ public class TestMasterNoCluster {
   public void testNotPullingDeadRegionServerFromZK()
       throws IOException, KeeperException, InterruptedException {
     final Configuration conf = TESTUTIL.getConfiguration();
-    final ServerName newServer = new ServerName("test.sample", 1, 101);
-    final ServerName deadServer = new ServerName("test.sample", 1, 100);
+    final ServerName newServer = ServerName.valueOf("test.sample", 1, 101);
+    final ServerName deadServer = ServerName.valueOf("test.sample", 1, 100);
     final MockRegionServer rs0 = new MockRegionServer(conf, newServer);
 
     HMaster master = new HMaster(conf) {

Modified: hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterStatusServlet.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterStatusServlet.java?rev=1543923&r1=1543922&r2=1543923&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterStatusServlet.java (original)
+++ hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterStatusServlet.java Wed Nov 20 20:06:16 2013
@@ -55,8 +55,8 @@ public class TestMasterStatusServlet {
   private Configuration conf;
   private HBaseAdmin admin;
 
-  static final ServerName FAKE_HOST = 
-    new ServerName("fakehost", 12345, 1234567890);
+  static final ServerName FAKE_HOST =
+      ServerName.valueOf("fakehost", 12345, 1234567890);
   static final HTableDescriptor FAKE_TABLE =
     new HTableDescriptor(TableName.valueOf("mytable"));
   static final HRegionInfo FAKE_HRI =
@@ -124,7 +124,7 @@ public class TestMasterStatusServlet {
     setupMockTables();
     
     new MasterStatusTmpl()
-      .setMetaLocation(new ServerName("metaserver:123,12345"))
+      .setMetaLocation(ServerName.valueOf("metaserver:123,12345"))
       .render(new StringWriter(),
         master, admin);
   }
@@ -134,16 +134,16 @@ public class TestMasterStatusServlet {
     setupMockTables();
     
     List<ServerName> servers = Lists.newArrayList(
-        new ServerName("rootserver:123,12345"),
-        new ServerName("metaserver:123,12345"));
+        ServerName.valueOf("rootserver:123,12345"),
+        ServerName.valueOf("metaserver:123,12345"));
     Set<ServerName> deadServers = new HashSet<ServerName>(
         Lists.newArrayList(
-        new ServerName("badserver:123,12345"),
-        new ServerName("uglyserver:123,12345"))
+            ServerName.valueOf("badserver:123,12345"),
+            ServerName.valueOf("uglyserver:123,12345"))
     );
 
     new MasterStatusTmpl()
-      .setMetaLocation(new ServerName("metaserver:123,12345"))
+      .setMetaLocation(ServerName.valueOf("metaserver:123,12345"))
       .setServers(servers)
       .setDeadServers(deadServers)
       .render(new StringWriter(),

Modified: hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java?rev=1543923&r1=1543922&r2=1543923&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java (original)
+++ hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java Wed Nov 20 20:06:16 2013
@@ -622,7 +622,7 @@ public class TestRegionPlacement {
             totalRegionNum.incrementAndGet();
             if (server != null) {
               ServerName serverName =
-                  new ServerName(Bytes.toString(server), -1);
+                  ServerName.valueOf(Bytes.toString(server), -1);
               if (favoredNodes != null) {
                 String placement = "[NOT FAVORED NODE]";
                 for (int i = 0; i < favoredServerList.length; i++) {

Modified: hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java?rev=1543923&r1=1543922&r2=1543923&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java (original)
+++ hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java Wed Nov 20 20:06:16 2013
@@ -66,7 +66,7 @@ public class TestRestartCluster {
     String unassignedZNode = zooKeeper.assignmentZNode;
     ZKUtil.createAndFailSilent(zooKeeper, unassignedZNode);
 
-    ServerName sn = new ServerName(HMaster.MASTER, -1, System.currentTimeMillis());
+    ServerName sn = ServerName.valueOf(HMaster.MASTER, -1, System.currentTimeMillis());
 
     ZKAssign.createNodeOffline(zooKeeper, HRegionInfo.FIRST_META_REGIONINFO, sn);
 

Modified: hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java?rev=1543923&r1=1543922&r2=1543923&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java (original)
+++ hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java Wed Nov 20 20:06:16 2013
@@ -77,7 +77,7 @@ import org.mockito.Mockito;
 @Category(MediumTests.class)
 public class TestSplitLogManager {
   private static final Log LOG = LogFactory.getLog(TestSplitLogManager.class);
-  private final ServerName DUMMY_MASTER = new ServerName("dummy-master,1,1");
+  private final ServerName DUMMY_MASTER = ServerName.valueOf("dummy-master,1,1");
   private final ServerManager sm = Mockito.mock(ServerManager.class);
   private final MasterServices master =  Mockito.mock(MasterServices.class);
 
@@ -271,9 +271,9 @@ public class TestSplitLogManager {
 
     String tasknode = submitTaskAndWait(batch, "foo/1");
     int version = ZKUtil.checkExists(zkw, tasknode);
-    final ServerName worker1 = new ServerName("worker1,1,1");
-    final ServerName worker2 = new ServerName("worker2,1,1");
-    final ServerName worker3 = new ServerName("worker3,1,1");
+    final ServerName worker1 = ServerName.valueOf("worker1,1,1");
+    final ServerName worker2 = ServerName.valueOf("worker2,1,1");
+    final ServerName worker3 = ServerName.valueOf("worker3,1,1");
     SplitLogTask slt = new SplitLogTask.Owned(worker1);
     ZKUtil.setData(zkw, tasknode, slt.toByteArray());
     waitForCounter(tot_mgr_heartbeat, 0, 1, to/2);
@@ -303,7 +303,7 @@ public class TestSplitLogManager {
 
     String tasknode = submitTaskAndWait(batch, "foo/1");
     int version = ZKUtil.checkExists(zkw, tasknode);
-    final ServerName worker1 = new ServerName("worker1,1,1");
+    final ServerName worker1 = ServerName.valueOf("worker1,1,1");
     SplitLogTask slt = new SplitLogTask.Owned(worker1);
     ZKUtil.setData(zkw, tasknode, slt.toByteArray());
     waitForCounter(tot_mgr_heartbeat, 0, 1, to/2);
@@ -330,7 +330,7 @@ public class TestSplitLogManager {
     slm = new SplitLogManager(zkw, conf, stopper, master, DUMMY_MASTER);
     TaskBatch batch = new TaskBatch();
     String tasknode = submitTaskAndWait(batch, "foo/1");
-    final ServerName worker1 = new ServerName("worker1,1,1");
+    final ServerName worker1 = ServerName.valueOf("worker1,1,1");
     SplitLogTask slt = new SplitLogTask.Done(worker1);
     ZKUtil.setData(zkw, tasknode, slt.toByteArray());
     synchronized (batch) {
@@ -351,7 +351,7 @@ public class TestSplitLogManager {
     TaskBatch batch = new TaskBatch();
 
     String tasknode = submitTaskAndWait(batch, "foo/1");
-    final ServerName worker1 = new ServerName("worker1,1,1");
+    final ServerName worker1 = ServerName.valueOf("worker1,1,1");
     SplitLogTask slt = new SplitLogTask.Err(worker1);
     ZKUtil.setData(zkw, tasknode, slt.toByteArray());
 
@@ -374,7 +374,7 @@ public class TestSplitLogManager {
     TaskBatch batch = new TaskBatch();
     String tasknode = submitTaskAndWait(batch, "foo/1");
     assertEquals(tot_mgr_resubmit.get(), 0);
-    final ServerName worker1 = new ServerName("worker1,1,1");
+    final ServerName worker1 = ServerName.valueOf("worker1,1,1");
     assertEquals(tot_mgr_resubmit.get(), 0);
     SplitLogTask slt = new SplitLogTask.Resigned(worker1);
     assertEquals(tot_mgr_resubmit.get(), 0);
@@ -398,7 +398,7 @@ public class TestSplitLogManager {
 
     // create an orphan task in OWNED state
     String tasknode1 = ZKSplitLog.getEncodedNodeName(zkw, "orphan/1");
-    final ServerName worker1 = new ServerName("worker1,1,1");
+    final ServerName worker1 = ServerName.valueOf("worker1,1,1");
     SplitLogTask slt = new SplitLogTask.Owned(worker1);
     zkw.getRecoverableZooKeeper().create(tasknode1, slt.toByteArray(), Ids.OPEN_ACL_UNSAFE,
         CreateMode.PERSISTENT);
@@ -413,7 +413,7 @@ public class TestSplitLogManager {
     // keep updating the orphan owned node every to/2 seconds
     for (int i = 0; i < (3 * to)/100; i++) {
       Thread.sleep(100);
-      final ServerName worker2 = new ServerName("worker1,1,1");
+      final ServerName worker2 = ServerName.valueOf("worker1,1,1");
       slt = new SplitLogTask.Owned(worker2);
       ZKUtil.setData(zkw, tasknode1, slt.toByteArray());
     }
@@ -437,7 +437,7 @@ public class TestSplitLogManager {
 
     String tasknode = submitTaskAndWait(batch, "foo/1");
     int version = ZKUtil.checkExists(zkw, tasknode);
-    final ServerName worker1 = new ServerName("worker1,1,1");
+    final ServerName worker1 = ServerName.valueOf("worker1,1,1");
     SplitLogTask slt = new SplitLogTask.Owned(worker1);
     ZKUtil.setData(zkw, tasknode, slt.toByteArray());
     if (tot_mgr_heartbeat.get() == 0) waitForCounter(tot_mgr_heartbeat, 0, 1, to/2);
@@ -461,7 +461,7 @@ public class TestSplitLogManager {
     TaskBatch batch = new TaskBatch();
 
     String tasknode = submitTaskAndWait(batch, "foo/1");
-    final ServerName worker1 = new ServerName("worker1,1,1");
+    final ServerName worker1 = ServerName.valueOf("worker1,1,1");
 
     SplitLogTask slt = new SplitLogTask.Owned(worker1);
     ZKUtil.setData(zkw, tasknode, slt.toByteArray());

Modified: hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestTableLockManager.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestTableLockManager.java?rev=1543923&r1=1543922&r2=1543923&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestTableLockManager.java (original)
+++ hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestTableLockManager.java Wed Nov 20 20:06:16 2013
@@ -268,7 +268,7 @@ public class TestTableLockManager {
   @Test(timeout = 600000)
   public void testReapAllTableLocks() throws Exception {
     prepareMiniZkCluster();
-    ServerName serverName = new ServerName("localhost:10000", 0);
+    ServerName serverName = ServerName.valueOf("localhost:10000", 0);
     final TableLockManager lockManager = TableLockManager.createTableLockManager(
         TEST_UTIL.getConfiguration(), TEST_UTIL.getZooKeeperWatcher(), serverName);
 

Modified: hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerTestBase.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerTestBase.java?rev=1543923&r1=1543922&r2=1543923&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerTestBase.java (original)
+++ hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerTestBase.java Wed Nov 20 20:06:16 2013
@@ -220,7 +220,7 @@ public class BalancerTestBase {
     String host = "srv" + rand.nextInt(100000);
     int port = rand.nextInt(60000);
     long startCode = rand.nextLong();
-    ServerName sn = new ServerName(host, port, startCode);
+    ServerName sn = ServerName.valueOf(host, port, startCode);
     return new ServerAndLoad(sn, numRegionsPerServer);
   }
 

Modified: hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestBaseLoadBalancer.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestBaseLoadBalancer.java?rev=1543923&r1=1543922&r2=1543923&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestBaseLoadBalancer.java (original)
+++ hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestBaseLoadBalancer.java Wed Nov 20 20:06:16 2013
@@ -149,7 +149,7 @@ public class TestBaseLoadBalancer extend
       // The old server would have had same host and port, but different
       // start code!
       ServerName snWithOldStartCode =
-          new ServerName(sn.getHostname(), sn.getPort(), sn.getStartcode() - 10);
+          ServerName.valueOf(sn.getHostname(), sn.getPort(), sn.getStartcode() - 10);
       existing.put(regions.get(i), snWithOldStartCode);
     }
     List<ServerName> listOfServerNames = getListOfServerNames(servers);

Modified: hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredNodeAssignmentHelper.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredNodeAssignmentHelper.java?rev=1543923&r1=1543922&r2=1543923&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredNodeAssignmentHelper.java (original)
+++ hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredNodeAssignmentHelper.java Wed Nov 20 20:06:16 2013
@@ -52,7 +52,7 @@ public class TestFavoredNodeAssignmentHe
     // Set up some server -> rack mappings
     // Have three racks in the cluster with 10 hosts each.
     for (int i = 0; i < 40; i++) {
-      ServerName server = new ServerName("foo"+i+":1234",-1);
+      ServerName server = ServerName.valueOf("foo" + i + ":1234", -1);
       if (i < 10) {
         Mockito.when(rackManager.getRack(server)).thenReturn("rack1");
         if (rackToServers.get("rack1") == null) {