You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2013/05/03 05:58:35 UTC

svn commit: r1478639 [1/10] - in /hbase/branches/0.95: hbase-client/src/main/java/org/apache/hadoop/hbase/ hbase-client/src/main/java/org/apache/hadoop/hbase/catalog/ hbase-client/src/main/java/org/apache/hadoop/hbase/client/ hbase-client/src/main/java...

Author: stack
Date: Fri May  3 03:58:33 2013
New Revision: 1478639

URL: http://svn.apache.org/r1478639
Log:
HBASE-8214 Remove proxy and engine, rely directly on pb generated Service

Added:
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectable.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionKey.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java.orig
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BadAuthException.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/StoppedRpcClientException.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/UnsupportedCellCodecException.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/UnsupportedCompressionCodecException.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/WrongVersionException.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SecurityInfo.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/EmptyServiceNameException.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServerInterface.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/UnknownServiceException.java
Removed:
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/IpcProtocol.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/MasterAdminProtocol.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/MasterMonitorProtocol.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/MasterProtocol.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AdminProtocol.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientProtocol.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseClient.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseClientRPC.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/ProtobufRpcClientEngine.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/ReflectionCache.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientEngine.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/security/KerberosInfo.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/security/TokenInfo.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/RegionServerStatusProtocol.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/HBaseServer.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/HBaseServerRPC.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ProtobufRpcServerEngine.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServerEngine.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/RandomTimeoutRpcEngine.java
Modified:
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/RemoteExceptionHandler.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/catalog/CatalogTracker.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnection.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterAdminKeepAliveConnection.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterMonitorKeepAliveConnection.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetaScanner.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ServerCallable.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionCoprocessorRpcChannel.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RemoteWithExtrasException.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKConfig.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
    hbase/branches/0.95/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromAdmin.java
    hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
    hbase/branches/0.95/hbase-common/src/test/java/org/apache/hadoop/hbase/ClassFinder.java
    hbase/branches/0.95/hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java
    hbase/branches/0.95/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestRebalanceAndKillServersTargeted.java
    hbase/branches/0.95/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestsDriver.java
    hbase/branches/0.95/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterAdminProtos.java
    hbase/branches/0.95/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterMonitorProtos.java
    hbase/branches/0.95/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/RPCProtos.java
    hbase/branches/0.95/hbase-protocol/src/main/protobuf/MasterAdmin.proto
    hbase/branches/0.95/hbase-protocol/src/main/protobuf/MasterMonitor.proto
    hbase/branches/0.95/hbase-protocol/src/main/protobuf/RPC.proto
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/ZNodeClearer.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/HFileSystem.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheConfig.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerWrapperImpl.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RequestContext.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ResponseFlag.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcCallContext.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/security/HBasePolicyProvider.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/security/HBaseSaslRpcServer.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenProvider.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HMerge.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseCluster.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestCatalogTracker.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaReaderEditor.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaReaderEditorNoCluster.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientScannerRPCTimeout.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWithScanLimits.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestChangingEncoding.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestDelayedRpc.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestHBaseClient.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestIPC.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestProtoBufRpc.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/protobuf/generated/TestDelayedRpcProtos.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesSplitRecovery.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManager.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestHMasterRPCException.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogWriter.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreSnapshotHelper.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockRegionServerServices.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java
    hbase/branches/0.95/hbase-server/src/test/protobuf/test_delayed_rpc.proto

Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/RemoteExceptionHandler.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/RemoteExceptionHandler.java?rev=1478639&r1=1478638&r2=1478639&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/RemoteExceptionHandler.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/RemoteExceptionHandler.java Fri May  3 03:58:33 2013
@@ -98,7 +98,6 @@ public class RemoteExceptionHandler {
 
       if (t instanceof IOException) {
         i = (IOException) t;
-
       } else {
         i = new IOException("server error");
         i.initCause(t);

Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/catalog/CatalogTracker.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/catalog/CatalogTracker.java?rev=1478639&r1=1478638&r2=1478639&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/catalog/CatalogTracker.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/catalog/CatalogTracker.java Fri May  3 03:58:33 2013
@@ -24,7 +24,6 @@ import org.apache.hadoop.conf.Configurat
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.client.AdminProtocol;
 import org.apache.hadoop.hbase.client.HConnection;
 import org.apache.hadoop.hbase.client.HConnectionManager;
 import org.apache.hadoop.hbase.client.HTable;
@@ -32,6 +31,7 @@ import org.apache.hadoop.hbase.client.Re
 import org.apache.hadoop.hbase.exceptions.NotAllMetaRegionsOnlineException;
 import org.apache.hadoop.hbase.exceptions.ServerNotRunningYetException;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.zookeeper.MetaRegionTracker;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
@@ -265,7 +265,7 @@ public class CatalogTracker {
    * @throws IOException
    * @deprecated Use #getMetaServerConnection(long)
    */
-  public AdminProtocol waitForMetaServerConnection(long timeout)
+  public AdminService.BlockingInterface waitForMetaServerConnection(long timeout)
   throws InterruptedException, NotAllMetaRegionsOnlineException, IOException {
     return getMetaServerConnection(timeout);
   }
@@ -281,7 +281,7 @@ public class CatalogTracker {
    * @throws NotAllMetaRegionsOnlineException if timed out waiting
    * @throws IOException
    */
-  AdminProtocol getMetaServerConnection(long timeout)
+  AdminService.BlockingInterface getMetaServerConnection(long timeout)
   throws InterruptedException, NotAllMetaRegionsOnlineException, IOException {
     return getCachedConnection(waitForMeta(timeout));
   }
@@ -313,14 +313,14 @@ public class CatalogTracker {
    * invocation, or may be null.
    * @throws IOException
    */
-  private AdminProtocol getCachedConnection(ServerName sn)
+  private AdminService.BlockingInterface getCachedConnection(ServerName sn)
   throws IOException {
     if (sn == null) {
       return null;
     }
-    AdminProtocol protocol = null;
+    AdminService.BlockingInterface service = null;
     try {
-      protocol = connection.getAdmin(sn);
+      service = 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.
@@ -349,7 +349,7 @@ public class CatalogTracker {
       }
 
     }
-    return protocol;
+    return service;
   }
 
   /**
@@ -367,7 +367,7 @@ public class CatalogTracker {
   // rather than have to pass it in.  Its made awkward by the fact that the
   // HRI is likely a proxy against remote server so the getServerName needs
   // to be fixed to go to a local method or to a cache before we can do this.
-  private boolean verifyRegionLocation(AdminProtocol hostingServer,
+  private boolean verifyRegionLocation(AdminService.BlockingInterface hostingServer,
       final ServerName address, final byte [] regionName)
   throws IOException {
     if (hostingServer == null) {
@@ -411,9 +411,9 @@ public class CatalogTracker {
    */
   public boolean verifyMetaRegionLocation(final long timeout)
   throws InterruptedException, IOException {
-    AdminProtocol connection = null;
+    AdminService.BlockingInterface service = null;
     try {
-      connection = waitForMetaServerConnection(timeout);
+      service = waitForMetaServerConnection(timeout);
     } catch (NotAllMetaRegionsOnlineException e) {
       // Pass
     } catch (ServerNotRunningYetException e) {
@@ -421,8 +421,8 @@ public class CatalogTracker {
     } catch (UnknownHostException e) {
       // Pass -- server name doesn't resolve so it can't be assigned anything.
     }
-    return (connection == null)? false:
-      verifyRegionLocation(connection,
+    return (service == null)? false:
+      verifyRegionLocation(service,
           this.metaRegionTracker.getMetaRegionLocation(), META_REGION_NAME);
   }
 

Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java?rev=1478639&r1=1478638&r2=1478639&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java Fri May  3 03:58:33 2013
@@ -318,8 +318,8 @@ public class ClientScanner extends Abstr
               if (retryAfterOutOfOrderException) {
                 retryAfterOutOfOrderException = false;
               } else {
-                throw new DoNotRetryIOException("Failed after retry"
-                    + ", it could be cause by rpc timeout", e);
+                throw new DoNotRetryIOException("Failed after retry of " +
+                  "OutOfOrderScannerNextException: was there a rpc timeout?", e);
               }
             }
             // Clear region

Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java?rev=1478639&r1=1478638&r2=1478639&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java Fri May  3 03:58:33 2013
@@ -68,6 +68,7 @@ import org.apache.hadoop.hbase.ipc.Maste
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.protobuf.ResponseConverter;
+import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionRequest;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionResponse;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionRequest;
@@ -78,6 +79,7 @@ import org.apache.hadoop.hbase.protobuf.
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerRequest;
+import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
@@ -570,7 +572,7 @@ public class HBaseAdmin implements Abort
           firstMetaServer.getRegionInfo().getRegionName(), scan, 1, true);
         Result[] values = null;
         // Get a batch at a time.
-        ClientProtocol server = connection.getClient(firstMetaServer.getServerName());
+        ClientService.BlockingInterface server = connection.getClient(firstMetaServer.getServerName());
         try {
           ScanResponse response = server.scan(null, request);
           values = ResponseConverter.getResults(response);
@@ -583,7 +585,7 @@ public class HBaseAdmin implements Abort
         if (values == null || values.length == 0) {
           tableExists = false;
           GetTableDescriptorsResponse htds;
-          MasterMonitorKeepAliveConnection master = connection.getKeepAliveMasterMonitor();
+          MasterMonitorKeepAliveConnection master = connection.getKeepAliveMasterMonitorService();
           try {
             GetTableDescriptorsRequest req =
               RequestConverter.buildGetTableDescriptorsRequest(null);
@@ -607,7 +609,7 @@ public class HBaseAdmin implements Abort
         if(tries == numRetries - 1) {           // no more tries left
           if (ex instanceof RemoteException) {
             throw ((RemoteException) ex).unwrapRemoteException();
-          }else {
+          } else {
             throw ex;
           }
         }
@@ -1221,7 +1223,7 @@ public class HBaseAdmin implements Abort
           "The servername cannot be null or empty.");
     }
     ServerName sn = new ServerName(serverName);
-    AdminProtocol admin = this.connection.getAdmin(sn);
+    AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
     // Close the region without updating zk state.
     CloseRegionRequest request =
       RequestConverter.buildCloseRegionRequest(encodedRegionName, false);
@@ -1246,8 +1248,7 @@ public class HBaseAdmin implements Abort
    */
   public void closeRegion(final ServerName sn, final HRegionInfo hri)
   throws IOException {
-    AdminProtocol admin =
-      this.connection.getAdmin(sn);
+    AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
     // Close the region without updating zk state.
     ProtobufUtil.closeRegion(admin, hri.getRegionName(), false);
   }
@@ -1257,8 +1258,7 @@ public class HBaseAdmin implements Abort
    */
   public List<HRegionInfo> getOnlineRegions(
       final ServerName sn) throws IOException {
-    AdminProtocol admin =
-      this.connection.getAdmin(sn);
+    AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
     return ProtobufUtil.getOnlineRegions(admin);
   }
 
@@ -1320,8 +1320,7 @@ public class HBaseAdmin implements Abort
 
   private void flush(final ServerName sn, final HRegionInfo hri)
   throws IOException {
-    AdminProtocol admin =
-      this.connection.getAdmin(sn);
+    AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
     FlushRegionRequest request =
       RequestConverter.buildFlushRegionRequest(hri.getRegionName());
     try {
@@ -1490,8 +1489,7 @@ public class HBaseAdmin implements Abort
   private void compact(final ServerName sn, final HRegionInfo hri,
       final boolean major, final byte [] family)
   throws IOException {
-    AdminProtocol admin =
-      this.connection.getAdmin(sn);
+    AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
     CompactRegionRequest request =
       RequestConverter.buildCompactRegionRequest(hri.getRegionName(), major, family);
     try {
@@ -1518,10 +1516,11 @@ public class HBaseAdmin implements Abort
    */
   public void move(final byte [] encodedRegionName, final byte [] destServerName)
   throws HBaseIOException, MasterNotRunningException, ZooKeeperConnectionException {
-    MasterAdminKeepAliveConnection master = connection.getKeepAliveMasterAdmin();
+    MasterAdminKeepAliveConnection stub = connection.getKeepAliveMasterAdminService();
     try {
-      MoveRegionRequest request = RequestConverter.buildMoveRegionRequest(encodedRegionName, destServerName);
-      master.moveRegion(null,request);
+      MoveRegionRequest request =
+        RequestConverter.buildMoveRegionRequest(encodedRegionName, destServerName);
+      stub.moveRegion(null,request);
     } catch (ServiceException se) {
       IOException ioe = ProtobufUtil.getRemoteException(se);
       if (ioe instanceof HBaseIOException) {
@@ -1530,9 +1529,8 @@ public class HBaseAdmin implements Abort
       LOG.error("Unexpected exception: " + se + " from calling HMaster.moveRegion");
     } catch (DeserializationException de) {
       LOG.error("Could not parse destination server name: " + de);
-    }
-    finally {
-      master.close();
+    } finally {
+      stub.close();
     }
   }
 
@@ -1587,7 +1585,7 @@ public class HBaseAdmin implements Abort
    */
   public void offline(final byte [] regionName)
   throws IOException {
-    MasterAdminKeepAliveConnection master = connection.getKeepAliveMasterAdmin();
+    MasterAdminKeepAliveConnection master = connection.getKeepAliveMasterAdminService();
     try {
       master.offlineRegion(null,RequestConverter.buildOfflineRegionRequest(regionName));
     } catch (ServiceException se) {
@@ -1605,11 +1603,11 @@ public class HBaseAdmin implements Abort
    */
   public boolean setBalancerRunning(final boolean on, final boolean synchronous)
   throws MasterNotRunningException, ZooKeeperConnectionException {
-    MasterAdminKeepAliveConnection master = connection.getKeepAliveMasterAdmin();
+    MasterAdminKeepAliveConnection stub = connection.getKeepAliveMasterAdminService();
     try {
       SetBalancerRunningRequest req =
         RequestConverter.buildSetBalancerRunningRequest(on, synchronous);
-      return master.setBalancerRunning(null, req).getPrevBalanceValue();
+      return stub.setBalancerRunning(null, req).getPrevBalanceValue();
     } catch (ServiceException se) {
       IOException ioe = ProtobufUtil.getRemoteException(se);
       if (ioe instanceof MasterNotRunningException) {
@@ -1623,7 +1621,7 @@ public class HBaseAdmin implements Abort
       // break interface by adding additional exception type.
       throw new MasterNotRunningException("Unexpected exception when calling balanceSwitch",se);
     } finally {
-      master.close();
+      stub.close();
     }
   }
 
@@ -1635,11 +1633,11 @@ public class HBaseAdmin implements Abort
    */
   public boolean balancer()
   throws MasterNotRunningException, ZooKeeperConnectionException, ServiceException {
-    MasterAdminKeepAliveConnection master = connection.getKeepAliveMasterAdmin();
+    MasterAdminKeepAliveConnection stub = connection.getKeepAliveMasterAdminService();
     try {
-      return master.balance(null,RequestConverter.buildBalanceRequest()).getBalancerRan();
+      return stub.balance(null,RequestConverter.buildBalanceRequest()).getBalancerRan();
     } finally {
-      master.close();
+      stub.close();
     }
   }
 
@@ -1652,12 +1650,12 @@ public class HBaseAdmin implements Abort
    */
   public boolean enableCatalogJanitor(boolean enable)
       throws ServiceException, MasterNotRunningException {
-    MasterAdminKeepAliveConnection master = connection.getKeepAliveMasterAdmin();
+    MasterAdminKeepAliveConnection stub = connection.getKeepAliveMasterAdminService();
     try {
-      return master.enableCatalogJanitor(null,
+      return stub.enableCatalogJanitor(null,
           RequestConverter.buildEnableCatalogJanitorRequest(enable)).getPrevValue();
     } finally {
-      master.close();
+      stub.close();
     }
   }
 
@@ -1668,12 +1666,12 @@ public class HBaseAdmin implements Abort
    * @throws MasterNotRunningException
    */
   public int runCatalogScan() throws ServiceException, MasterNotRunningException {
-    MasterAdminKeepAliveConnection master = connection.getKeepAliveMasterAdmin();
+    MasterAdminKeepAliveConnection stub = connection.getKeepAliveMasterAdminService();
     try {
-      return master.runCatalogScan(null,
+      return stub.runCatalogScan(null,
           RequestConverter.buildCatalogScanRequest()).getScanResult();
     } finally {
-      master.close();
+      stub.close();
     }
   }
 
@@ -1683,12 +1681,12 @@ public class HBaseAdmin implements Abort
    * @throws org.apache.hadoop.hbase.exceptions.MasterNotRunningException
    */
   public boolean isCatalogJanitorEnabled() throws ServiceException, MasterNotRunningException {
-    MasterAdminKeepAliveConnection master = connection.getKeepAliveMasterAdmin();
+    MasterAdminKeepAliveConnection stub = connection.getKeepAliveMasterAdminService();
     try {
-      return master.isCatalogJanitorEnabled(null,
+      return stub.isCatalogJanitorEnabled(null,
           RequestConverter.buildIsCatalogJanitorEnabledRequest()).getValue();
     } finally {
-      master.close();
+      stub.close();
     }
   }
 
@@ -1704,7 +1702,7 @@ public class HBaseAdmin implements Abort
       final byte[] encodedNameOfRegionB, final boolean forcible)
       throws IOException {
     MasterAdminKeepAliveConnection master = connection
-        .getKeepAliveMasterAdmin();
+        .getKeepAliveMasterAdminService();
     try {
       DispatchMergingRegionsRequest request = RequestConverter
           .buildDispatchMergingRegionsRequest(encodedNameOfRegionA,
@@ -1800,8 +1798,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);
+    AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
     ProtobufUtil.split(admin, hri, splitPoint);
   }
 
@@ -1924,7 +1921,7 @@ public class HBaseAdmin implements Abort
   throws IOException {
     String hostname = Addressing.parseHostname(hostnamePort);
     int port = Addressing.parsePort(hostnamePort);
-    AdminProtocol admin =
+    AdminService.BlockingInterface admin =
       this.connection.getAdmin(new ServerName(hostname, port, 0));
     StopServerRequest request = RequestConverter.buildStopServerRequest(
       "Called by admin client " + this.connection.toString());
@@ -2067,7 +2064,7 @@ 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);
+    AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
     RollWALWriterRequest request = RequestConverter.buildRollWALWriterRequest();
     try {
       RollWALWriterResponse response = admin.rollWALWriter(null, request);
@@ -2127,8 +2124,7 @@ public class HBaseAdmin implements Abort
           throw new NoServerForRegionException(Bytes.toStringBinary(tableNameOrRegionName));
         } else {
           ServerName sn = regionServerPair.getSecond();
-          AdminProtocol admin =
-            this.connection.getAdmin(sn);
+          AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
           GetRegionInfoRequest request = RequestConverter.buildGetRegionInfoRequest(
             regionServerPair.getFirst().getRegionName(), true);
           GetRegionInfoResponse response = admin.getRegionInfo(null, request);
@@ -2143,8 +2139,7 @@ public class HBaseAdmin implements Abort
           if (pair.getSecond() == null) continue;
           try {
             ServerName sn = pair.getSecond();
-            AdminProtocol admin =
-              this.connection.getAdmin(sn);
+            AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
             GetRegionInfoRequest request = RequestConverter.buildGetRegionInfoRequest(
               pair.getFirst().getRegionName(), true);
             GetRegionInfoResponse response = admin.getRegionInfo(null, request);
@@ -2607,7 +2602,7 @@ public class HBaseAdmin implements Abort
    * Create a {@link MasterAdminCallable} to use it.
    */
   private <V> V execute(MasterAdminCallable<V> function) throws IOException {
-    function.masterAdmin = connection.getKeepAliveMasterAdmin();
+    function.masterAdmin = connection.getKeepAliveMasterAdminService();
     try {
       return executeCallable(function);
     } finally {
@@ -2621,7 +2616,7 @@ public class HBaseAdmin implements Abort
    * Create a {@link MasterAdminCallable} to use it.
    */
   private <V> V execute(MasterMonitorCallable<V> function) throws IOException {
-    function.masterMonitor = connection.getKeepAliveMasterMonitor();
+    function.masterMonitor = connection.getKeepAliveMasterMonitorService();
     try {
       return executeCallable(function);
     } finally {

Added: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectable.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectable.java?rev=1478639&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectable.java (added)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectable.java Fri May  3 03:58:33 2013
@@ -0,0 +1,48 @@
+/**
+ * Copyright The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * This class makes it convenient for one to execute a command in the context
+ * of a {@link HConnection} instance based on the given {@link Configuration}.
+ *
+ * <p>
+ * If you find yourself wanting to use a {@link HConnection} for a relatively
+ * short duration of time, and do not want to deal with the hassle of creating
+ * and cleaning up that resource, then you should consider using this
+ * convenience class.
+ *
+ * @param <T>
+ *          the return type of the {@link HConnectable#connect(HConnection)}
+ *          method.
+ */
+public abstract class HConnectable<T> {
+  public Configuration conf;
+
+  protected HConnectable(Configuration conf) {
+    this.conf = conf;
+  }
+
+  public abstract T connect(HConnection connection) throws IOException;
+}
\ No newline at end of file

Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnection.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnection.java?rev=1478639&r1=1478638&r2=1478639&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnection.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnection.java Fri May  3 03:58:33 2013
@@ -18,40 +18,43 @@
  */
 package org.apache.hadoop.hbase.client;
 
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.MasterAdminProtocol;
-import org.apache.hadoop.hbase.MasterMonitorProtocol;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.catalog.CatalogTracker;
 import org.apache.hadoop.hbase.client.coprocessor.Batch;
 import org.apache.hadoop.hbase.exceptions.MasterNotRunningException;
 import org.apache.hadoop.hbase.exceptions.ZooKeeperConnectionException;
+import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
+import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService;
+import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MasterAdminService;
+import org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.MasterMonitorService;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 
-import java.io.Closeable;
-import java.io.IOException;
-import java.util.List;
-import java.util.concurrent.ExecutorService;
-
 /**
- * Cluster connection.  Hosts a connection to the ZooKeeper ensemble and
- * thereafter into the HBase cluster.  Knows how to locate regions out on the cluster,
+ * A cluster connection.  Knows how to find the master, locate regions out on the cluster,
  * keeps a cache of locations and then knows how to recalibrate after they move.
- * {@link HConnectionManager} manages instances of this class.
+ * {@link HConnectionManager} manages instances of this class.   This is NOT a connection to a
+ * particular server but to all servers in the cluster.  An implementation takes care of individual
+ * connections at a lower level.
  *
  * <p>HConnections are used by {@link HTable} mostly but also by
  * {@link HBaseAdmin}, {@link CatalogTracker},
  * and {@link ZooKeeperWatcher}.  HConnection instances can be shared.  Sharing
  * is usually what you want because rather than each HConnection instance
  * having to do its own discovery of regions out on the cluster, instead, all
- * clients get to share the one cache of locations.  Sharing makes cleanup of
- * HConnections awkward.  See {@link HConnectionManager} for cleanup
- * discussion.
+ * clients get to share the one cache of locations.  {@link HConnectionManager} does the
+ * sharing for you if you go by it getting connections.  Sharing makes cleanup of
+ * HConnections awkward.  See {@link HConnectionManager} for cleanup discussion.
  *
  * @see HConnectionManager
  */
@@ -213,29 +216,14 @@ public interface HConnection extends Abo
       final boolean offlined) throws IOException;
 
   /**
-   * Returns a {@link MasterAdminProtocol} to the active master
+   * Returns a {@link MasterAdminKeepAliveConnection} to the active master
    */
-  public MasterAdminProtocol getMasterAdmin() throws IOException;
+  public MasterAdminService.BlockingInterface getMasterAdmin() throws IOException;
 
   /**
-   * Returns an {@link MasterMonitorProtocol} to the active master
+   * Returns an {@link MasterMonitorKeepAliveConnection} to the active master
    */
-  public MasterMonitorProtocol getMasterMonitor() throws IOException;
-
-
-  /**
-   * Establishes a connection to the region server at the specified address.
-   * @param hostname RegionServer hostname
-   * @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;
-
+  public MasterMonitorService.BlockingInterface getMasterMonitor() throws IOException;
 
   /**
    * Establishes a connection to the region server at the specified address.
@@ -243,27 +231,10 @@ public interface HConnection extends Abo
    * @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.
-   *
-   * @param hostname RegionServer hostname
-   * @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;
-
+  public AdminService.BlockingInterface getAdmin(final ServerName serverName) throws IOException;
 
   /**
-   * Establishes a connection to the region server at the specified address, and return
+   * Establishes a connection to the region server at the specified address, and returns
    * a region client protocol.
    *
    * @param serverName
@@ -271,30 +242,17 @@ public interface HConnection extends Abo
    * @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
-   * @param port RegionServer port
-   * @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.
-   */
-  @Deprecated
-  public AdminProtocol getAdmin(final String hostname, final int port, boolean getMaster)
-  throws IOException;
+  public ClientService.BlockingInterface getClient(final ServerName serverName) throws IOException;
 
   /**
    * Establishes a connection to the region server at the specified address.
    * @param serverName
-   * @param getMaster - do we check if master is alive
+   * @param getMaster do we check if master is alive
    * @return proxy for HRegionServer
    * @throws IOException if a remote or network exception occurs
+   * @deprecated You can pass master flag but nothing special is done.
    */
-  public AdminProtocol getAdmin(final ServerName serverName, boolean getMaster)
+  public AdminService.BlockingInterface getAdmin(final ServerName serverName, boolean getMaster)
       throws IOException;
 
   /**
@@ -417,13 +375,14 @@ public interface HConnection extends Abo
   public void clearCaches(final ServerName sn);
 
   /**
-   * This function allows HBaseAdminProtocol and potentially others to get a shared MasterMonitor
+   * This function allows HBaseAdmin and potentially others to get a shared MasterMonitor
    * connection.
    * @return The shared instance. Never returns null.
    * @throws MasterNotRunningException
    */
-  public MasterMonitorKeepAliveConnection getKeepAliveMasterMonitor()
-      throws MasterNotRunningException;
+  // TODO: Why is this in the public interface when the returned type is shutdown package access?
+  public MasterMonitorKeepAliveConnection getKeepAliveMasterMonitorService()
+  throws MasterNotRunningException;
 
   /**
    * This function allows HBaseAdmin and potentially others to get a shared MasterAdminProtocol
@@ -431,7 +390,8 @@ public interface HConnection extends Abo
    * @return The shared instance. Never returns null.
    * @throws MasterNotRunningException
    */
-  public MasterAdminKeepAliveConnection getKeepAliveMasterAdmin() throws MasterNotRunningException;
+  // TODO: Why is this in the public interface when the returned type is shutdown package access?
+  public MasterAdminKeepAliveConnection getKeepAliveMasterAdminService() throws MasterNotRunningException;
 
   /**
    * @param serverName
@@ -439,4 +399,3 @@ public interface HConnection extends Abo
    */
   public boolean isDeadServer(ServerName serverName);
 }
-

Added: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionKey.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionKey.java?rev=1478639&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionKey.java (added)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionKey.java Fri May  3 03:58:33 2013
@@ -0,0 +1,140 @@
+/**
+ * Copyright The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.security.User;
+
+/**
+ * Denotes a unique key to an {@link HConnection} instance.
+ *
+ * In essence, this class captures the properties in {@link Configuration}
+ * that may be used in the process of establishing a connection. In light of
+ * that, if any new such properties are introduced into the mix, they must be
+ * added to the {@link HConnectionKey#properties} list.
+ *
+ */
+class HConnectionKey {
+  final static String[] CONNECTION_PROPERTIES = new String[] {
+      HConstants.ZOOKEEPER_QUORUM, HConstants.ZOOKEEPER_ZNODE_PARENT,
+      HConstants.ZOOKEEPER_CLIENT_PORT,
+      HConstants.ZOOKEEPER_RECOVERABLE_WAITTIME,
+      HConstants.HBASE_CLIENT_PAUSE, HConstants.HBASE_CLIENT_RETRIES_NUMBER,
+      HConstants.HBASE_RPC_TIMEOUT_KEY,
+      HConstants.HBASE_CLIENT_PREFETCH_LIMIT,
+      HConstants.HBASE_META_SCANNER_CACHING,
+      HConstants.HBASE_CLIENT_INSTANCE_ID };
+
+  private Map<String, String> properties;
+  private String username;
+
+  HConnectionKey(Configuration conf) {
+    Map<String, String> m = new HashMap<String, String>();
+    if (conf != null) {
+      for (String property : CONNECTION_PROPERTIES) {
+        String value = conf.get(property);
+        if (value != null) {
+          m.put(property, value);
+        }
+      }
+    }
+    this.properties = Collections.unmodifiableMap(m);
+
+    try {
+      User currentUser = User.getCurrent();
+      if (currentUser != null) {
+        username = currentUser.getName();
+      }
+    } catch (IOException ioe) {
+      HConnectionManager.LOG.warn("Error obtaining current user, skipping username in HConnectionKey", ioe);
+    }
+  }
+
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    if (username != null) {
+      result = username.hashCode();
+    }
+    for (String property : CONNECTION_PROPERTIES) {
+      String value = properties.get(property);
+      if (value != null) {
+        result = prime * result + value.hashCode();
+      }
+    }
+
+    return result;
+  }
+
+
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings (value="ES_COMPARING_STRINGS_WITH_EQ",
+      justification="Optimization")
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj)
+      return true;
+    if (obj == null)
+      return false;
+    if (getClass() != obj.getClass())
+      return false;
+    HConnectionKey that = (HConnectionKey) obj;
+    if (this.username != null && !this.username.equals(that.username)) {
+      return false;
+    } else if (this.username == null && that.username != null) {
+      return false;
+    }
+    if (this.properties == null) {
+      if (that.properties != null) {
+        return false;
+      }
+    } else {
+      if (that.properties == null) {
+        return false;
+      }
+      for (String property : CONNECTION_PROPERTIES) {
+        String thisValue = this.properties.get(property);
+        String thatValue = that.properties.get(property);
+        //noinspection StringEquality
+        if (thisValue == thatValue) {
+          continue;
+        }
+        if (thisValue == null || !thisValue.equals(thatValue)) {
+          return false;
+        }
+      }
+    }
+    return true;
+  }
+
+  @Override
+  public String toString() {
+    return "HConnectionKey{" +
+      "properties=" + properties +
+      ", username='" + username + '\'' +
+      '}';
+  }
+}
\ No newline at end of file