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/02/16 15:13:47 UTC

svn commit: r1446897 - in /hbase/trunk: hbase-it/src/test/java/org/apache/hadoop/hbase/ hbase-server/src/main/java/org/apache/hadoop/hbase/ hbase-server/src/main/java/org/apache/hadoop/hbase/catalog/ hbase-server/src/main/java/org/apache/hadoop/hbase/c...

Author: nkeywal
Date: Sat Feb 16 14:13:46 2013
New Revision: 1446897

URL: http://svn.apache.org/r1446897
Log:
HBASE-7861 Use the ServerName in the Connection#getClient and Connection#getAdmin code

Modified:
    hbase/trunk/hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/HRegionLocation.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/RegionMovedException.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/catalog/CatalogTracker.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HConnection.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ServerCallable.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHRegionLocation.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestCatalogTracker.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaReaderEditorNoCluster.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesSplitRecovery.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java

Modified: hbase/trunk/hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java?rev=1446897&r1=1446896&r2=1446897&view=diff
==============================================================================
--- hbase/trunk/hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java (original)
+++ hbase/trunk/hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java Sat Feb 16 14:13:46 2013
@@ -85,12 +85,12 @@ public class DistributedHBaseCluster ext
 
   @Override
   public AdminProtocol getAdminProtocol(ServerName serverName) throws IOException {
-    return admin.getConnection().getAdmin(serverName.getHostname(), serverName.getPort());
+    return admin.getConnection().getAdmin(serverName);
   }
 
   @Override
   public ClientProtocol getClientProtocol(ServerName serverName) throws IOException {
-    return admin.getConnection().getClient(serverName.getHostname(), serverName.getPort());
+    return admin.getConnection().getClient(serverName);
   }
 
   @Override
@@ -193,7 +193,7 @@ public class DistributedHBaseCluster ext
       return null;
     }
 
-    AdminProtocol client = connection.getAdmin(regionLoc.getHostname(), regionLoc.getPort());
+    AdminProtocol client = connection.getAdmin(regionLoc.getServerName());
     ServerInfo info = ProtobufUtil.getServerInfo(client);
     return ProtobufUtil.toServerName(info.getServerName());
   }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/HRegionLocation.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/HRegionLocation.java?rev=1446897&r1=1446896&r2=1446897&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/HRegionLocation.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/HRegionLocation.java Sat Feb 16 14:13:46 2013
@@ -33,42 +33,27 @@ import org.apache.hadoop.hbase.util.Addr
 @InterfaceStability.Evolving
 public class HRegionLocation implements Comparable<HRegionLocation> {
   private final HRegionInfo regionInfo;
-  private final String hostname;
-  private final int port;
+  private final ServerName serverName;
   private final long seqNum;
   // Cache of the 'toString' result.
   private String cachedString = null;
   // Cache of the hostname + port
   private String cachedHostnamePort;
 
-  /**
-   * Constructor
-   * @param regionInfo the HRegionInfo for the region
-   */
-  public HRegionLocation(HRegionInfo regionInfo, final String hostname,
-      final int port, final long seqNum) {
+  public HRegionLocation(HRegionInfo regionInfo, ServerName serverName, long seqNum) {
     this.regionInfo = regionInfo;
-    this.hostname = hostname;
-    this.port = port;
+    this.serverName = serverName;
     this.seqNum = seqNum;
   }
 
   /**
-   * Test constructor w/o seqNum.
-   */
-  public HRegionLocation(HRegionInfo regionInfo, final String hostname, final int port) {
-    this(regionInfo, hostname, port, 0);
-  }
-
-  /**
    * @see java.lang.Object#toString()
    */
   @Override
   public synchronized String toString() {
     if (this.cachedString == null) {
       this.cachedString = "region=" + this.regionInfo.getRegionNameAsString() +
-      ", hostname=" + this.hostname + ", port=" + this.port
-      + ", seqNum=" + seqNum;
+      ", hostname=" + this.serverName + ", seqNum=" + seqNum;
     }
     return this.cachedString;
   }
@@ -95,9 +80,7 @@ public class HRegionLocation implements 
    */
   @Override
   public int hashCode() {
-    int result = this.hostname.hashCode();
-    result ^= this.port;
-    return result;
+    return this.serverName.hashCode();
   }
 
   /** @return HRegionInfo */
@@ -106,11 +89,11 @@ public class HRegionLocation implements 
   }
 
   public String getHostname() {
-    return this.hostname;
+    return this.serverName.getHostname();
   }
 
   public int getPort() {
-    return this.port;
+    return this.serverName.getPort();
   }
 
   public long getSeqNum() {
@@ -123,18 +106,16 @@ public class HRegionLocation implements 
   public synchronized String getHostnamePort() {
     if (this.cachedHostnamePort == null) {
       this.cachedHostnamePort =
-        Addressing.createHostAndPortStr(this.hostname, this.port);
+        Addressing.createHostAndPortStr(this.getHostname(), this.getPort());
     }
     return this.cachedHostnamePort;
   }
 
-  //
-  // Comparable
-  //
+  public ServerName getServerName() {
+    return serverName;
+  }
 
   public int compareTo(HRegionLocation o) {
-    int result = this.hostname.compareTo(o.getHostname());
-    if (result != 0) return result;
-    return this.port - o.getPort();
+    return serverName.compareTo(o.getServerName());
   }
 }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/RegionMovedException.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/RegionMovedException.java?rev=1446897&r1=1446896&r2=1446897&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/RegionMovedException.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/RegionMovedException.java Sat Feb 16 14:13:46 2013
@@ -35,17 +35,19 @@ public class RegionMovedException extend
 
   private final String hostname;
   private final int port;
+  private final long startCode;
   private final long locationSeqNum;
 
   private static final String HOST_FIELD = "hostname=";
   private static final String PORT_FIELD = "port=";
+  private static final String STARTCODE_FIELD = "startCode=";
   private static final String LOCATIONSEQNUM_FIELD = "locationSeqNum=";
 
-  public RegionMovedException(final String hostname, final int port,
-    final long locationSeqNum) {
-    super();
-    this.hostname = hostname;
-    this.port = port;
+
+  public RegionMovedException(ServerName serverName, long locationSeqNum) {
+    this.hostname = serverName.getHostname();
+    this.port = serverName.getPort();
+    this.startCode = serverName.getStartcode();
     this.locationSeqNum = locationSeqNum;
   }
 
@@ -57,6 +59,10 @@ public class RegionMovedException extend
     return port;
   }
 
+  public ServerName getServerName(){
+    return new ServerName(hostname, port, startCode);
+  }
+
   public long getLocationSeqNum() {
     return locationSeqNum;
   }
@@ -69,22 +75,27 @@ public class RegionMovedException extend
   public RegionMovedException(String s) {
     int posHostname = s.indexOf(HOST_FIELD) + HOST_FIELD.length();
     int posPort = s.indexOf(PORT_FIELD) + PORT_FIELD.length();
+    int posStartCode = s.indexOf(STARTCODE_FIELD) + STARTCODE_FIELD.length();
     int posSeqNum = s.indexOf(LOCATIONSEQNUM_FIELD) + LOCATIONSEQNUM_FIELD.length();
 
     String tmpHostname = null;
     int tmpPort = -1;
+    long tmpStartCode = -1;
     long tmpSeqNum = HConstants.NO_SEQNUM;
     try {
       // TODO: this whole thing is extremely brittle.
       tmpHostname = s.substring(posHostname, s.indexOf(' ', posHostname));
-      tmpPort = Integer.parseInt(s.substring(posPort, s.indexOf('.', posPort)));
+      tmpPort = Integer.parseInt(s.substring(posPort, s.indexOf(' ', posPort)));
+      tmpStartCode =  Long.parseLong(s.substring(posStartCode, s.indexOf('.', posStartCode)));
       tmpSeqNum = Long.parseLong(s.substring(posSeqNum, s.indexOf('.', posSeqNum)));
     } catch (Exception ignored) {
-      LOG.warn("Can't parse the hostname and the port from this string: " + s + ", continuing");
+      LOG.warn("Can't parse the hostname, port and startCode from this string: " +
+          s + ", continuing");
     }
 
     hostname = tmpHostname;
     port = tmpPort;
+    startCode = tmpStartCode;
     locationSeqNum = tmpSeqNum;
   }
 
@@ -92,8 +103,8 @@ public class RegionMovedException extend
   public String getMessage() {
     // TODO: deserialization above depends on this. That is bad, but also means this
     // should be modified carefully.
-    return "Region moved to: " + HOST_FIELD + hostname + " " + PORT_FIELD + port + ". As of "
-      + LOCATIONSEQNUM_FIELD + locationSeqNum + ".";
+    return "Region moved to: " + HOST_FIELD + hostname + " " + PORT_FIELD + port + " " +
+        STARTCODE_FIELD + startCode + ". As of "  + LOCATIONSEQNUM_FIELD + locationSeqNum + ".";
   }
 
   /**

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/catalog/CatalogTracker.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/catalog/CatalogTracker.java?rev=1446897&r1=1446896&r2=1446897&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/catalog/CatalogTracker.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/catalog/CatalogTracker.java Sat Feb 16 14:13:46 2013
@@ -497,7 +497,7 @@ public class CatalogTracker {
     }
     AdminProtocol protocol = null;
     try {
-      protocol = connection.getAdmin(sn.getHostname(), sn.getPort());
+      protocol = connection.getAdmin(sn);
     } catch (RetriesExhaustedException e) {
       if (e.getCause() != null && e.getCause() instanceof ConnectException) {
         // Catch this; presume it means the cached connection has gone bad.

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java?rev=1446897&r1=1446896&r2=1446897&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java Sat Feb 16 14:13:46 2013
@@ -539,7 +539,7 @@ public class HBaseAdmin implements Abort
 
     // Wait until all regions deleted
     ClientProtocol server =
-      connection.getClient(firstMetaServer.getHostname(), firstMetaServer.getPort());
+      connection.getClient(firstMetaServer.getServerName());
     for (int tries = 0; tries < (this.numRetries * this.retryLongerMultiplier); tries++) {
       try {
 
@@ -1153,8 +1153,7 @@ public class HBaseAdmin implements Abort
           "The servername cannot be null or empty.");
     }
     ServerName sn = new ServerName(serverName);
-    AdminProtocol admin = this.connection.getAdmin(
-        sn.getHostname(), sn.getPort());
+    AdminProtocol admin = this.connection.getAdmin(sn);
     // Close the region without updating zk state.
     CloseRegionRequest request =
       RequestConverter.buildCloseRegionRequest(encodedRegionName, false);
@@ -1180,7 +1179,7 @@ public class HBaseAdmin implements Abort
   public void closeRegion(final ServerName sn, final HRegionInfo hri)
   throws IOException {
     AdminProtocol admin =
-      this.connection.getAdmin(sn.getHostname(), sn.getPort());
+      this.connection.getAdmin(sn);
     // Close the region without updating zk state.
     ProtobufUtil.closeRegion(admin, hri.getRegionName(), false);
   }
@@ -1191,7 +1190,7 @@ public class HBaseAdmin implements Abort
   public List<HRegionInfo> getOnlineRegions(
       final ServerName sn) throws IOException {
     AdminProtocol admin =
-      this.connection.getAdmin(sn.getHostname(), sn.getPort());
+      this.connection.getAdmin(sn);
     return ProtobufUtil.getOnlineRegions(admin);
   }
 
@@ -1254,7 +1253,7 @@ public class HBaseAdmin implements Abort
   private void flush(final ServerName sn, final HRegionInfo hri)
   throws IOException {
     AdminProtocol admin =
-      this.connection.getAdmin(sn.getHostname(), sn.getPort());
+      this.connection.getAdmin(sn);
     FlushRegionRequest request =
       RequestConverter.buildFlushRegionRequest(hri.getRegionName());
     try {
@@ -1424,7 +1423,7 @@ public class HBaseAdmin implements Abort
       final boolean major, final byte [] family)
   throws IOException {
     AdminProtocol admin =
-      this.connection.getAdmin(sn.getHostname(), sn.getPort());
+      this.connection.getAdmin(sn);
     CompactRegionRequest request =
       RequestConverter.buildCompactRegionRequest(hri.getRegionName(), major, family);
     try {
@@ -1702,7 +1701,7 @@ public class HBaseAdmin implements Abort
   private void split(final ServerName sn, final HRegionInfo hri,
       byte[] splitPoint) throws IOException {
     AdminProtocol admin =
-      this.connection.getAdmin(sn.getHostname(), sn.getPort());
+      this.connection.getAdmin(sn);
     ProtobufUtil.split(admin, hri, splitPoint);
   }
 
@@ -1826,7 +1825,7 @@ public class HBaseAdmin implements Abort
     String hostname = Addressing.parseHostname(hostnamePort);
     int port = Addressing.parsePort(hostnamePort);
     AdminProtocol admin =
-      this.connection.getAdmin(hostname, port);
+      this.connection.getAdmin(new ServerName(hostname, port, 0));
     StopServerRequest request = RequestConverter.buildStopServerRequest(
       "Called by admin client " + this.connection.toString());
     try {
@@ -1836,6 +1835,7 @@ public class HBaseAdmin implements Abort
     }
   }
 
+
   /**
    * @return cluster status
    * @throws IOException if a remote or network exception occurs
@@ -1967,9 +1967,8 @@ public class HBaseAdmin implements Abort
  public synchronized  byte[][] rollHLogWriter(String serverName)
       throws IOException, FailedLogCloseException {
     ServerName sn = new ServerName(serverName);
-    AdminProtocol admin = this.connection.getAdmin(
-        sn.getHostname(), sn.getPort());
-    RollWALWriterRequest request = RequestConverter.buildRollWALWriterRequest();;
+    AdminProtocol admin = this.connection.getAdmin(sn);
+    RollWALWriterRequest request = RequestConverter.buildRollWALWriterRequest();
     try {
       RollWALWriterResponse response = admin.rollWALWriter(null, request);
       int regionCount = response.getRegionToFlushCount();
@@ -2029,7 +2028,7 @@ public class HBaseAdmin implements Abort
         } else {
           ServerName sn = regionServerPair.getSecond();
           AdminProtocol admin =
-            this.connection.getAdmin(sn.getHostname(), sn.getPort());
+            this.connection.getAdmin(sn);
           GetRegionInfoRequest request = RequestConverter.buildGetRegionInfoRequest(
             regionServerPair.getFirst().getRegionName(), true);
           GetRegionInfoResponse response = admin.getRegionInfo(null, request);
@@ -2045,7 +2044,7 @@ public class HBaseAdmin implements Abort
           try {
             ServerName sn = pair.getSecond();
             AdminProtocol admin =
-              this.connection.getAdmin(sn.getHostname(), sn.getPort());
+              this.connection.getAdmin(sn);
             GetRegionInfoRequest request = RequestConverter.buildGetRegionInfoRequest(
               pair.getFirst().getRegionName(), true);
             GetRegionInfoResponse response = admin.getRegionInfo(null, request);

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HConnection.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HConnection.java?rev=1446897&r1=1446896&r2=1446897&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HConnection.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HConnection.java Sat Feb 16 14:13:46 2013
@@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.HTableDes
 import org.apache.hadoop.hbase.MasterAdminProtocol;
 import org.apache.hadoop.hbase.MasterMonitorProtocol;
 import org.apache.hadoop.hbase.MasterNotRunningException;
+import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
 import org.apache.hadoop.hbase.catalog.CatalogTracker;
 import org.apache.hadoop.hbase.client.coprocessor.Batch;
@@ -208,11 +209,23 @@ public interface HConnection extends Abo
    * @param port RegionServer port
    * @return proxy for HRegionServer
    * @throws IOException if a remote or network exception occurs
-   *
+   * @deprecated - use @link {#getAdmin(final ServerName serverName)} which takes into account
+   *  the startCode
    */
+  @Deprecated
   public AdminProtocol getAdmin(final String hostname, final int port)
   throws IOException;
 
+
+  /**
+   * Establishes a connection to the region server at the specified address.
+   * @param serverName
+   * @return proxy for HRegionServer
+   * @throws IOException if a remote or network exception occurs
+   */
+  public AdminProtocol getAdmin(final ServerName serverName)
+      throws IOException;
+
   /**
    * Establishes a connection to the region server at the specified address, and return
    * a region client protocol.
@@ -221,11 +234,25 @@ public interface HConnection extends Abo
    * @param port RegionServer port
    * @return ClientProtocol proxy for RegionServer
    * @throws IOException if a remote or network exception occurs
-   *
+   * @deprecated - use @link {#getClient(final ServerName serverName)} which takes into account
+   *  the startCode
    */
+  @Deprecated
   public ClientProtocol getClient(final String hostname, final int port)
   throws IOException;
 
+
+  /**
+   * Establishes a connection to the region server at the specified address, and return
+   * a region client protocol.
+   *
+   * @param serverName
+   * @return ClientProtocol proxy for RegionServer
+   * @throws IOException if a remote or network exception occurs
+   *
+   */
+  public ClientProtocol getClient(final ServerName serverName) throws IOException;
+
   /**
    * Establishes a connection to the region server at the specified address.
    * @param hostname RegionServer hostname
@@ -233,12 +260,24 @@ public interface HConnection extends Abo
    * @param getMaster - do we check if master is alive
    * @return proxy for HRegionServer
    * @throws IOException if a remote or network exception occurs
+   * @deprecated use @link {#getAdmin(final ServerName serverName, boolean getMaster)}
+   * which takes into account the startCode.
    */
-  public AdminProtocol getAdmin(final String hostname,
-     final int port, boolean getMaster)
+  @Deprecated
+  public AdminProtocol getAdmin(final String hostname, final int port, boolean getMaster)
   throws IOException;
 
   /**
+   * Establishes a connection to the region server at the specified address.
+   * @param serverName
+   * @param getMaster - do we check if master is alive
+   * @return proxy for HRegionServer
+   * @throws IOException if a remote or network exception occurs
+   */
+  public AdminProtocol getAdmin(final ServerName serverName, boolean getMaster)
+      throws IOException;
+
+  /**
    * Find region location hosting passed row
    * @param tableName table name
    * @param row Row to find.

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java?rev=1446897&r1=1446896&r2=1446897&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java Sat Feb 16 14:13:46 2013
@@ -85,7 +85,6 @@ import org.apache.hadoop.hbase.protobuf.
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.Addressing;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.SoftValueSortedMap;
 import org.apache.hadoop.hbase.util.Triple;
@@ -959,8 +958,7 @@ public class HConnectionManager {
           LOG.debug("Looked up root region location, connection=" + this +
             "; serverName=" + ((servername == null) ? "null" : servername));
           if (servername == null) return null;
-          return new HRegionLocation(HRegionInfo.ROOT_REGIONINFO, servername.getHostname(),
-              servername.getPort(), 0);
+          return new HRegionLocation(HRegionInfo.ROOT_REGIONINFO, servername, 0);
         } catch (InterruptedException e) {
           Thread.currentThread().interrupt();
           return null;
@@ -1008,8 +1006,8 @@ public class HConnectionManager {
               return true; // don't cache it
             }
             // instantiate the location
-            HRegionLocation loc = new HRegionLocation(regionInfo, serverName.getHostname(),
-                serverName.getPort(), HRegionInfo.getSeqNumDuringOpen(result));
+            HRegionLocation loc = new HRegionLocation(regionInfo, serverName,
+                HRegionInfo.getSeqNumDuringOpen(result));
             // cache this meta entry
             cacheLocation(tableName, null, loc);
             return true;
@@ -1063,7 +1061,7 @@ public class HConnectionManager {
           // If null still, go around again.
           if (metaLocation == null) continue;
           ClientProtocol server =
-            getClient(metaLocation.getHostname(), metaLocation.getPort());
+            getClient(metaLocation.getServerName());
 
           Result regionInfoRow = null;
           // This block guards against two threads trying to load the meta
@@ -1133,8 +1131,8 @@ public class HConnectionManager {
           }
 
           // Instantiate the location
-          location = new HRegionLocation(regionInfo, serverName.getHostname(),
-                  serverName.getPort(), HRegionInfo.getSeqNumDuringOpen(regionInfoRow));
+          location = new HRegionLocation(regionInfo, serverName,
+              HRegionInfo.getSeqNumDuringOpen(regionInfoRow));
           cacheLocation(tableName, null, location);
           return location;
         } catch (TableNotFoundException e) {
@@ -1218,7 +1216,7 @@ public class HConnectionManager {
         return possibleRegion;
       }
 
-      // Passed all the way through, so we got nothin - complete cache miss
+      // Passed all the way through, so we got nothing - complete cache miss
       return null;
     }
 
@@ -1368,24 +1366,46 @@ public class HConnectionManager {
     }
 
     @Override
-    public AdminProtocol getAdmin(final String hostname,
-        final int port) throws IOException {
-      return getAdmin(hostname, port, false);
+    @Deprecated
+    public AdminProtocol getAdmin(final String hostname, final int port) throws IOException {
+      return getAdmin(new ServerName(hostname, port, 0L));
+    }
+
+    @Override
+    public AdminProtocol getAdmin(final ServerName serverName)
+        throws IOException {
+      return getAdmin(serverName, false);
     }
 
     @Override
+    @Deprecated
     public ClientProtocol getClient(final String hostname, final int port)
     throws IOException {
       return (ClientProtocol)getProtocol(hostname, port, clientClass);
     }
 
     @Override
+    public ClientProtocol getClient(final ServerName serverName)
+        throws IOException {
+      return (ClientProtocol)
+          getProtocol(serverName.getHostname(), serverName.getPort(), clientClass);
+    }
+
+    @Override
+    @Deprecated
     public AdminProtocol getAdmin(final String hostname, final int port,
         final boolean master)
     throws IOException {
       return (AdminProtocol)getProtocol(hostname, port, adminClass);
     }
 
+    @Override
+    public AdminProtocol getAdmin(final ServerName serverName, final boolean master)
+        throws IOException {
+      return (AdminProtocol)getProtocol(
+          serverName.getHostname(), serverName.getPort(), adminClass);
+    }
+
     /**
      * Either the passed <code>isa</code> is null or <code>hostname</code>
      * can be but not both.
@@ -1757,8 +1777,7 @@ public class HConnectionManager {
 
               @Override
               public void connect(boolean reload) throws IOException {
-                server = connection.getClient(
-                  loc.getHostname(), loc.getPort());
+                server = connection.getClient(loc.getServerName());
               }
             };
           return callable.withoutRetries();
@@ -1767,8 +1786,8 @@ public class HConnectionManager {
    }
 
    void updateCachedLocation(HRegionInfo hri, HRegionLocation source,
-       String hostname, int port, long seqNum) {
-      HRegionLocation newHrl = new HRegionLocation(hri, hostname, port, seqNum);
+       ServerName serverName, long seqNum) {
+      HRegionLocation newHrl = new HRegionLocation(hri, serverName, seqNum);
       synchronized (this.cachedRegionLocations) {
         cacheLocation(hri.getTableName(), source, newHrl);
       }
@@ -1781,7 +1800,7 @@ public class HConnectionManager {
     */
     void deleteCachedLocation(HRegionInfo hri, HRegionLocation source) {
       boolean isStaleDelete = false;
-      HRegionLocation oldLocation = null;
+      HRegionLocation oldLocation;
       synchronized (this.cachedRegionLocations) {
         Map<byte[], HRegionLocation> tableLocations =
           getTableLocations(hri.getTableName());
@@ -1829,7 +1848,7 @@ public class HConnectionManager {
         LOG.info("Region " + regionInfo.getRegionNameAsString() + " moved to " +
           rme.getHostname() + ":" + rme.getPort() + " according to " + source.getHostnamePort());
         updateCachedLocation(
-            regionInfo, source, rme.getHostname(), rme.getPort(), rme.getLocationSeqNum());
+            regionInfo, source, rme.getServerName(), rme.getLocationSeqNum());
       } else {
         deleteCachedLocation(regionInfo, source);
       }
@@ -2124,8 +2143,7 @@ public class HConnectionManager {
           }
         }
 
-        public String getDescriptionAndClear()
-        {
+        public String getDescriptionAndClear(){
           if (exceptions.isEmpty()) {
             return "";
           }
@@ -2134,7 +2152,7 @@ public class HConnectionManager {
           actions.clear();
           addresses.clear();
           return result;
-        };
+        }
 
         private RetriesExhaustedWithDetailsException makeException() {
           return new RetriesExhaustedWithDetailsException(exceptions, actions, addresses);

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ServerCallable.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ServerCallable.java?rev=1446897&r1=1446896&r2=1446897&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ServerCallable.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ServerCallable.java Sat Feb 16 14:13:46 2013
@@ -87,8 +87,7 @@ public abstract class ServerCallable<T> 
    */
   public void connect(final boolean reload) throws IOException {
     this.location = connection.getRegionLocation(tableName, row, reload);
-    this.server = connection.getClient(location.getHostname(),
-        location.getPort());
+    this.server = connection.getClient(location.getServerName());
   }
 
   /** @return the server name

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java?rev=1446897&r1=1446896&r2=1446897&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java Sat Feb 16 14:13:46 2013
@@ -683,7 +683,7 @@ public class ServerManager {
     AdminProtocol admin = this.serverConnections.get(sn);
     if (admin == null) {
       LOG.debug("New connection to " + sn.toString());
-      admin = this.connection.getAdmin(sn.getHostname(), sn.getPort());
+      admin = this.connection.getAdmin(sn);
       this.serverConnections.put(sn, admin);
     }
     return admin;

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java?rev=1446897&r1=1446896&r2=1446897&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java Sat Feb 16 14:13:46 2013
@@ -2527,8 +2527,7 @@ public class HRegionServer implements Cl
     if (region == null) {
       MovedRegionInfo moveInfo = getMovedRegion(encodedRegionName);
       if (moveInfo != null) {
-        throw new RegionMovedException(moveInfo.getServerName().getHostname(),
-            moveInfo.getServerName().getPort(), moveInfo.getSeqNum());
+        throw new RegionMovedException(moveInfo.getServerName(), moveInfo.getSeqNum());
       } else {
         throw new NotServingRegionException("Region is not online: " + encodedRegionName);
       }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java?rev=1446897&r1=1446896&r2=1446897&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java Sat Feb 16 14:13:46 2013
@@ -764,7 +764,7 @@ public class ReplicationSource extends T
     }
     ServerName address =
         currentPeers.get(random.nextInt(this.currentPeers.size()));
-    return this.conn.getAdmin(address.getHostname(), address.getPort());
+    return this.conn.getAdmin(address);
   }
 
   /**

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java?rev=1446897&r1=1446896&r2=1446897&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java Sat Feb 16 14:13:46 2013
@@ -3031,7 +3031,7 @@ public class HBaseFsck extends Configure
       errors.progress();
       try {
         AdminProtocol server =
-          connection.getAdmin(rsinfo.getHostname(), rsinfo.getPort());
+          connection.getAdmin(rsinfo);
 
         // list all online regions from this region server
         List<HRegionInfo> regions = ProtobufUtil.getOnlineRegions(server);

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java?rev=1446897&r1=1446896&r2=1446897&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java Sat Feb 16 14:13:46 2013
@@ -149,7 +149,7 @@ public class HBaseFsckRepair {
   public static void closeRegionSilentlyAndWait(HBaseAdmin admin,
       ServerName server, HRegionInfo region) throws IOException, InterruptedException {
     HConnection connection = admin.getConnection();
-    AdminProtocol rs = connection.getAdmin(server.getHostname(), server.getPort());
+    AdminProtocol rs = connection.getAdmin(server);
     ProtobufUtil.closeRegion(rs, region.getRegionName(), false);
     long timeout = admin.getConfiguration()
       .getLong("hbase.hbck.close.timeout", 120000);

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHRegionLocation.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHRegionLocation.java?rev=1446897&r1=1446896&r2=1446897&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHRegionLocation.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHRegionLocation.java Sat Feb 16 14:13:46 2013
@@ -37,24 +37,24 @@ public class TestHRegionLocation {
   public void testHashAndEqualsCode() {
     ServerName hsa1 = new ServerName("localhost", 1234, -1L);
     HRegionLocation hrl1 = new HRegionLocation(HRegionInfo.FIRST_META_REGIONINFO,
-      hsa1.getHostname(), hsa1.getPort());
+        hsa1, HConstants.NO_SEQNUM);
     HRegionLocation hrl2 = new HRegionLocation(HRegionInfo.FIRST_META_REGIONINFO,
-      hsa1.getHostname(), hsa1.getPort());
+        hsa1, HConstants.NO_SEQNUM);
     assertEquals(hrl1.hashCode(), hrl2.hashCode());
     assertTrue(hrl1.equals(hrl2));
     HRegionLocation hrl3 = new HRegionLocation(HRegionInfo.ROOT_REGIONINFO,
-      hsa1.getHostname(), hsa1.getPort());
+        hsa1, HConstants.NO_SEQNUM);
     assertNotSame(hrl1, hrl3);
     // 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);
     HRegionLocation hrl4 = new HRegionLocation(HRegionInfo.ROOT_REGIONINFO,
-        hsa2.getHostname(), hsa2.getPort());
+        hsa2, HConstants.NO_SEQNUM);
     // These have same HRI but different locations so should be different.
     assertFalse(hrl3.equals(hrl4));
     HRegionLocation hrl5 = new HRegionLocation(hrl4.getRegionInfo(),
-        hrl4.getHostname(), hrl4.getPort(), hrl4.getSeqNum() + 1);
+        hrl4.getServerName(), hrl4.getSeqNum() + 1);
     assertTrue(hrl4.equals(hrl5));
   }
 
@@ -62,7 +62,7 @@ public class TestHRegionLocation {
   public void testToString() {
     ServerName hsa1 = new ServerName("localhost", 1234, -1L);
     HRegionLocation hrl1 = new HRegionLocation(HRegionInfo.FIRST_META_REGIONINFO,
-      hsa1.getHostname(), hsa1.getPort());
+      hsa1, HConstants.NO_SEQNUM);
     System.out.println(hrl1.toString());
   }
 
@@ -70,10 +70,10 @@ public class TestHRegionLocation {
   public void testCompareTo() {
     ServerName hsa1 = new ServerName("localhost", 1234, -1L);
     HRegionLocation hsl1 =
-      new HRegionLocation(HRegionInfo.ROOT_REGIONINFO, hsa1.getHostname(), hsa1.getPort());
+      new HRegionLocation(HRegionInfo.ROOT_REGIONINFO, hsa1, HConstants.NO_SEQNUM);
     ServerName hsa2 = new ServerName("localhost", 1235, -1L);
     HRegionLocation hsl2 =
-      new HRegionLocation(HRegionInfo.ROOT_REGIONINFO, hsa2.getHostname(), hsa2.getPort());
+      new HRegionLocation(HRegionInfo.ROOT_REGIONINFO, hsa2, HConstants.NO_SEQNUM);
     assertTrue(hsl1.compareTo(hsl1) == 0);
     assertTrue(hsl2.compareTo(hsl2) == 0);
     int compare1 = hsl1.compareTo(hsl2);

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestCatalogTracker.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestCatalogTracker.java?rev=1446897&r1=1446896&r2=1446897&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestCatalogTracker.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestCatalogTracker.java Sat Feb 16 14:13:46 2013
@@ -339,8 +339,7 @@ public class TestCatalogTracker {
       Mockito.mock(AdminProtocol.class);
     Mockito.when(implementation.getRegionInfo((RpcController)Mockito.any(),
       (GetRegionInfoRequest)Mockito.any())).thenThrow(connectException);
-    Mockito.when(connection.getAdmin(Mockito.anyString(),
-      Mockito.anyInt(), Mockito.anyBoolean())).
+    Mockito.when(connection.getAdmin(Mockito.any(ServerName.class), Mockito.anyBoolean())).
       thenReturn(implementation);
     final CatalogTracker ct = constructAndStartCatalogTracker(connection);
     try {
@@ -475,8 +474,8 @@ public class TestCatalogTracker {
    * {@link HConnection#getConfiguration()} is called, a 'location' when
    * {@link HConnection#getRegionLocation(byte[], byte[], boolean)} is called,
    * and that returns the passed {@link AdminProtocol} instance when
-   * {@link HConnection#getAdmin(String, int)} is called, returns the passed
-   * {@link ClientProtocol} instance when {@link HConnection#getClient(String, int)}
+   * {@link HConnection#getAdmin(ServerName)} is called, returns the passed
+   * {@link ClientProtocol} instance when {@link HConnection#getClient(ServerName)}
    * is called (Be sure call
    * {@link HConnectionManager#deleteConnection(org.apache.hadoop.conf.Configuration)}
    * when done with this mocked Connection.
@@ -489,8 +488,7 @@ public class TestCatalogTracker {
     Mockito.doNothing().when(connection).close();
     // Make it so we return any old location when asked.
     final HRegionLocation anyLocation =
-      new HRegionLocation(HRegionInfo.FIRST_META_REGIONINFO, SN.getHostname(),
-        SN.getPort());
+      new HRegionLocation(HRegionInfo.FIRST_META_REGIONINFO, SN, HConstants.NO_SEQNUM);
     Mockito.when(connection.getRegionLocation((byte[]) Mockito.any(),
         (byte[]) Mockito.any(), Mockito.anyBoolean())).
       thenReturn(anyLocation);
@@ -499,12 +497,12 @@ public class TestCatalogTracker {
       thenReturn(anyLocation);
     if (admin != null) {
       // If a call to getHRegionConnection, return this implementation.
-      Mockito.when(connection.getAdmin(Mockito.anyString(), Mockito.anyInt())).
+      Mockito.when(connection.getAdmin(Mockito.any(ServerName.class))).
         thenReturn(admin);
     }
     if (client != null) {
       // If a call to getClient, return this implementation.
-      Mockito.when(connection.getClient(Mockito.anyString(), Mockito.anyInt())).
+      Mockito.when(connection.getClient(Mockito.any(ServerName.class))).
         thenReturn(client);
     }
     return connection;

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaReaderEditorNoCluster.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaReaderEditorNoCluster.java?rev=1446897&r1=1446896&r2=1446897&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaReaderEditorNoCluster.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaReaderEditorNoCluster.java Sat Feb 16 14:13:46 2013
@@ -120,7 +120,7 @@ public class TestMetaReaderEditorNoClust
   /**
    * Test that MetaReader will ride over server throwing
    * "Server not running" IOEs.
-   * @see https://issues.apache.org/jira/browse/HBASE-3446
+   * @see @link {https://issues.apache.org/jira/browse/HBASE-3446}
    * @throws IOException
    * @throws InterruptedException
    */
@@ -133,7 +133,7 @@ public class TestMetaReaderEditorNoClust
     // This is a servername we use in a few places below.
     ServerName sn = new ServerName("example.com", 1234, System.currentTimeMillis());
 
-    HConnection connection = null;
+    HConnection connection;
     CatalogTracker ct = null;
     try {
       // Mock an ClientProtocol. Our mock implementation will fail a few
@@ -178,8 +178,7 @@ public class TestMetaReaderEditorNoClust
       // Fix the location lookup so it 'works' though no network.  First
       // make an 'any location' object.
       final HRegionLocation anyLocation =
-        new HRegionLocation(HRegionInfo.FIRST_META_REGIONINFO, sn.getHostname(),
-          sn.getPort());
+        new HRegionLocation(HRegionInfo.FIRST_META_REGIONINFO, sn, HConstants.NO_SEQNUM);
       // Return the any location object when locateRegion is called in HTable
       // constructor and when its called by ServerCallable (it uses getRegionLocation).
       // The ugly format below comes of 'Important gotcha on spying real objects!' from
@@ -192,7 +191,7 @@ public class TestMetaReaderEditorNoClust
 
       // Now shove our HRI implementation into the spied-upon connection.
       Mockito.doReturn(implementation).
-        when(connection).getClient(Mockito.anyString(), Mockito.anyInt());
+        when(connection).getClient(Mockito.any(ServerName.class));
 
       // Now start up the catalogtracker with our doctored Connection.
       ct = new CatalogTracker(zkw, null, connection, ABORTABLE);

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java?rev=1446897&r1=1446896&r2=1446897&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java Sat Feb 16 14:13:46 2013
@@ -20,12 +20,11 @@ package org.apache.hadoop.hbase.client;
 import java.io.IOException;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
-import org.apache.hadoop.hbase.client.AdminProtocol;
-import org.apache.hadoop.hbase.client.ClientProtocol;
 import org.apache.hadoop.hbase.client.HConnectionManager.HConnectionImplementation;
 import org.apache.hadoop.hbase.client.HConnectionManager.HConnectionKey;
 import org.mockito.Mockito;
@@ -44,7 +43,7 @@ public class HConnectionTestingUtility {
    * configuration instance.  Minimally the mock will return
    * <code>conf</conf> when {@link HConnection#getConfiguration()} is invoked.
    * Be sure to shutdown the connection when done by calling
-   * {@link HConnectionManager#deleteConnection(Configuration, boolean)} else it
+   * {@link HConnectionManager#deleteConnection(HConnectionKey, boolean)} else it
    * will stick around; this is probably not what you want.
    * @param conf configuration
    * @return HConnection object for <code>conf</code>
@@ -70,7 +69,7 @@ public class HConnectionTestingUtility {
    * more of the popular {@link HConnection} methods so they do 'normal'
    * operation (see return doc below for list). Be sure to shutdown the
    * connection when done by calling
-   * {@link HConnectionManager#deleteConnection(Configuration, boolean)} else it
+   * {@link HConnectionManager#deleteConnection(HConnectionKey, boolean)} else it
    * will stick around; this is probably not what you want.
    *
    * @param conf Configuration to use
@@ -86,10 +85,10 @@ public class HConnectionTestingUtility {
    * {@link HConnection#getConfiguration()} is called, a 'location' when
    * {@link HConnection#getRegionLocation(byte[], byte[], boolean)} is called,
    * and that returns the passed {@link AdminProtocol} instance when
-   * {@link HConnection#getAdmin(String, int)} is called, returns the passed
-   * {@link ClientProtocol} instance when {@link HConnection#getClient(String, int)}
+   * {@link HConnection#getAdmin(ServerName)} is called, returns the passed
+   * {@link ClientProtocol} instance when {@link HConnection#getClient(ServerName)}
    * is called (Be sure call
-   * {@link HConnectionManager#deleteConnection(org.apache.hadoop.conf.Configuration, boolean)}
+   * {@link HConnectionManager#deleteConnection(HConnectionKey, boolean)}
    * when done with this mocked Connection.
    * @throws IOException
    */
@@ -100,7 +99,7 @@ public class HConnectionTestingUtility {
     HConnection c = HConnectionTestingUtility.getMockedConnection(conf);
     Mockito.doNothing().when(c).close();
     // Make it so we return a particular location when asked.
-    final HRegionLocation loc = new HRegionLocation(hri, sn.getHostname(), sn.getPort());
+    final HRegionLocation loc = new HRegionLocation(hri, sn, HConstants.NO_SEQNUM);
     Mockito.when(c.getRegionLocation((byte[]) Mockito.any(),
         (byte[]) Mockito.any(), Mockito.anyBoolean())).
       thenReturn(loc);
@@ -108,12 +107,12 @@ public class HConnectionTestingUtility {
       thenReturn(loc);
     if (admin != null) {
       // If a call to getAdmin, return this implementation.
-      Mockito.when(c.getAdmin(Mockito.anyString(), Mockito.anyInt())).
+      Mockito.when(c.getAdmin(Mockito.any(ServerName.class))).
         thenReturn(admin);
     }
     if (client != null) {
       // If a call to getClient, return this client.
-      Mockito.when(c.getClient(Mockito.anyString(), Mockito.anyInt())).
+      Mockito.when(c.getClient(Mockito.any(ServerName.class))).
         thenReturn(client);
     }
     return c;
@@ -123,12 +122,13 @@ public class HConnectionTestingUtility {
    * Get a Mockito spied-upon {@link HConnection} that goes with the passed
    * <code>conf</code> configuration instance.
    * Be sure to shutdown the connection when done by calling
-   * {@link HConnectionManager#deleteConnection(Configuration, boolean)} else it
+   * {@link HConnectionManager#deleteConnection(HConnectionKey, boolean)} else it
    * will stick around; this is probably not what you want.
    * @param conf configuration
    * @return HConnection object for <code>conf</code>
    * @throws ZooKeeperConnectionException
-   * @see http://mockito.googlecode.com/svn/branches/1.6/javadoc/org/mockito/Mockito.html#spy(T)
+   * @see @link
+   * {http://mockito.googlecode.com/svn/branches/1.6/javadoc/org/mockito/Mockito.html#spy(T)}
    */
   public static HConnection getSpiedConnection(final Configuration conf)
   throws ZooKeeperConnectionException {

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java?rev=1446897&r1=1446896&r2=1446897&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java Sat Feb 16 14:13:46 2013
@@ -115,8 +115,7 @@ public class TestFromClientSide3 {
     HConnection conn = HConnectionManager.getConnection(TEST_UTIL
         .getConfiguration());
     HRegionLocation loc = table.getRegionLocation(row, true);
-    AdminProtocol server = conn.getAdmin(loc.getHostname(), loc
-        .getPort());
+    AdminProtocol server = conn.getAdmin(loc.getServerName());
     byte[] regName = loc.getRegionInfo().getRegionName();
 
     for (int i = 0; i < nFlushes; i++) {
@@ -163,8 +162,7 @@ public class TestFromClientSide3 {
     // Verify we have multiple store files.
     HRegionLocation loc = hTable.getRegionLocation(row, true);
     byte[] regionName = loc.getRegionInfo().getRegionName();
-    AdminProtocol server = connection.getAdmin(
-      loc.getHostname(), loc.getPort());
+    AdminProtocol server = connection.getAdmin(loc.getServerName());
     assertTrue(ProtobufUtil.getStoreFiles(
       server, regionName, FAMILY).size() > 1);
 
@@ -177,7 +175,7 @@ public class TestFromClientSide3 {
       loc = hTable.getRegionLocation(row, true);
       if (!loc.getRegionInfo().isOffline()) {
         regionName = loc.getRegionInfo().getRegionName();
-        server = connection.getAdmin(loc.getHostname(), loc.getPort());
+        server = connection.getAdmin(loc.getServerName());
         if (ProtobufUtil.getStoreFiles(
             server, regionName, FAMILY).size() <= 1) {
           break;
@@ -211,7 +209,7 @@ public class TestFromClientSide3 {
     Thread.sleep(10 * 1000);
     loc = hTable.getRegionLocation(row, true);
     regionName = loc.getRegionInfo().getRegionName();
-    server = connection.getAdmin(loc.getHostname(), loc.getPort());
+    server = connection.getAdmin(loc.getServerName());
     int sfCount = ProtobufUtil.getStoreFiles(
       server, regionName, FAMILY).size();
     assertTrue(sfCount > 1);
@@ -236,8 +234,7 @@ public class TestFromClientSide3 {
       loc = hTable.getRegionLocation(row, true);
       regionName = loc.getRegionInfo().getRegionName();
       try {
-        server = connection.getAdmin(loc.getHostname(), loc
-            .getPort());
+        server = connection.getAdmin(loc.getServerName());
         if (ProtobufUtil.getStoreFiles(
             server, regionName, FAMILY).size() < sfCount) {
           break;

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java?rev=1446897&r1=1446896&r2=1446897&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java Sat Feb 16 14:13:46 2013
@@ -151,8 +151,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, "127.0.0.1", nextPort,
-      HConstants.LATEST_TIMESTAMP);
+    conn.updateCachedLocation(loc.getRegionInfo(), loc, new ServerName("127.0.0.1", nextPort,
+      HConstants.LATEST_TIMESTAMP), HConstants.LATEST_TIMESTAMP);
     Assert.assertEquals(conn.getCachedLocation(TABLE_NAME, ROW).getPort(), nextPort);
 
     conn.forceDeleteCachedLocation(TABLE_NAME.clone(), ROW.clone());
@@ -349,34 +349,34 @@ public class TestHCM {
     HRegionLocation location = conn.getCachedLocation(TABLE_NAME2, ROW);
     assertNotNull(location);
 
-    HRegionLocation anySource = new HRegionLocation(location.getRegionInfo(),
-        location.getHostname(), location.getPort() - 1);
+    HRegionLocation anySource = new HRegionLocation(location.getRegionInfo(), new ServerName(
+        location.getHostname(), location.getPort() - 1, 0L), HConstants.NO_SEQNUM);
 
     // Same server as already in cache reporting - overwrites any value despite seqNum.
     int nextPort = location.getPort() + 1;
     conn.updateCachedLocation(location.getRegionInfo(), location,
-        "127.0.0.1", nextPort, location.getSeqNum() - 1);
+        new ServerName("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,
-        "127.0.0.1", nextPort, location.getSeqNum() - 1);
+        new ServerName("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,
-        "127.0.0.1", nextPort, location.getSeqNum() + 1);
+        new ServerName("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,
-        "127.0.0.1", nextPort, location.getSeqNum() - 1);
+        new ServerName("127.0.0.1", nextPort, 0), location.getSeqNum() - 1);
     location = conn.getCachedLocation(TABLE_NAME2, ROW);
     Assert.assertEquals(nextPort - 1, location.getPort());
   }

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesSplitRecovery.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesSplitRecovery.java?rev=1446897&r1=1446896&r2=1446897&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesSplitRecovery.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesSplitRecovery.java Sat Feb 16 14:13:46 2013
@@ -37,10 +37,12 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.LargeTests;
+import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableExistsException;
 import org.apache.hadoop.hbase.client.ClientProtocol;
 import org.apache.hadoop.hbase.client.HConnection;
@@ -269,7 +271,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,
-        "example.org", 1234);
+        new ServerName("example.org", 1234, 0), HConstants.NO_SEQNUM);
     Mockito.when(c.getRegionLocation((byte[]) Mockito.any(),
         (byte[]) Mockito.any(), Mockito.anyBoolean())).
       thenReturn(loc);
@@ -278,7 +280,7 @@ public class TestLoadIncrementalHFilesSp
     ClientProtocol hri = Mockito.mock(ClientProtocol.class);
     Mockito.when(hri.bulkLoadHFile((RpcController)Mockito.any(), (BulkLoadHFileRequest)Mockito.any())).
       thenThrow(new ServiceException(new IOException("injecting bulk load error")));
-    Mockito.when(c.getClient(Mockito.anyString(), Mockito.anyInt())).
+    Mockito.when(c.getClient(Mockito.any(ServerName.class))).
       thenReturn(hri);
     return c;
   }

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java?rev=1446897&r1=1446896&r2=1446897&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java Sat Feb 16 14:13:46 2013
@@ -165,8 +165,7 @@ public class TestHRegionServerBulkLoad {
           public Void call() throws Exception {
             LOG.debug("compacting " + location + " for row "
                 + Bytes.toStringBinary(row));
-            AdminProtocol server = connection.getAdmin(
-              location.getHostname(), location.getPort());
+            AdminProtocol server = connection.getAdmin(location.getServerName());
             CompactRegionRequest request =
               RequestConverter.buildCompactRegionRequest(
                 location.getRegionInfo().getRegionName(), true, null);

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java?rev=1446897&r1=1446896&r2=1446897&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java Sat Feb 16 14:13:46 2013
@@ -505,7 +505,7 @@ public class TestHBaseFsck {
     HConnection connection = admin.getConnection();
     for (ServerName hsi : regionServers) {
       AdminProtocol server =
-        connection.getAdmin(hsi.getHostname(), hsi.getPort());
+        connection.getAdmin(hsi);
 
       // list all online regions from this region server
       List<HRegionInfo> regions = ProtobufUtil.getOnlineRegions(server);