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 [10/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/jav...

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestHMasterRPCException.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestHMasterRPCException.java?rev=1478639&r1=1478638&r2=1478639&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestHMasterRPCException.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestHMasterRPCException.java Fri May  3 03:58:33 2013
@@ -22,19 +22,19 @@ package org.apache.hadoop.hbase.master;
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
-import java.net.InetSocketAddress;
 import java.net.SocketTimeoutException;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.*;
-import org.apache.hadoop.hbase.ipc.HBaseClientRPC;
-import org.apache.hadoop.hbase.MasterMonitorProtocol;
-import org.apache.hadoop.hbase.ipc.ProtobufRpcClientEngine;
+import org.apache.hadoop.hbase.ipc.RpcClient;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningRequest;
+import org.apache.hadoop.hbase.security.User;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import com.google.protobuf.BlockingRpcChannel;
 import com.google.protobuf.ServiceException;
 
 @Category(MediumTests.class)
@@ -46,29 +46,31 @@ public class TestHMasterRPCException {
     TEST_UTIL.startMiniZKCluster();
     Configuration conf = TEST_UTIL.getConfiguration();
     conf.set(HConstants.MASTER_PORT, "0");
-
     HMaster hm = new HMaster(conf);
-
     ServerName sm = hm.getServerName();
-    InetSocketAddress isa = new InetSocketAddress(sm.getHostname(), sm.getPort());
-    ProtobufRpcClientEngine engine =
-        new ProtobufRpcClientEngine(conf, HConstants.CLUSTER_ID_DEFAULT);
+    RpcClient rpcClient = new RpcClient(conf, HConstants.CLUSTER_ID_DEFAULT);
     try {
       int i = 0;
       //retry the RPC a few times; we have seen SocketTimeoutExceptions if we
       //try to connect too soon. Retry on SocketTimeoutException.
       while (i < 20) {
         try {
-          MasterMonitorProtocol inf = engine.getProxy(
-              MasterMonitorProtocol.class, isa, conf, 100 * 10);
-          inf.isMasterRunning(null, IsMasterRunningRequest.getDefaultInstance());
+          BlockingRpcChannel channel =
+            rpcClient.createBlockingRpcChannel(sm, User.getCurrent(), 0);
+          MasterMonitorProtos.MasterMonitorService.BlockingInterface stub =
+            MasterMonitorProtos.MasterMonitorService.newBlockingStub(channel);
+          stub.isMasterRunning(null, IsMasterRunningRequest.getDefaultInstance());
           fail();
         } catch (ServiceException ex) {
           IOException ie = ProtobufUtil.getRemoteException(ex);
           if (!(ie instanceof SocketTimeoutException)) {
-            if(ie.getMessage().startsWith(
-                "org.apache.hadoop.hbase.exceptions.ServerNotRunningYetException: Server is not running yet")) {
+            if (ie.getMessage().startsWith("org.apache.hadoop.hbase.exceptions." +
+                "ServerNotRunningYetException: Server is not running yet")) {
+              // Done.  Got the exception we wanted.
+              System.out.println("Expected exception: " + ie.getMessage());
               return;
+            } else {
+              throw ex;
             }
           } else {
             System.err.println("Got SocketTimeoutException. Will retry. ");
@@ -81,7 +83,7 @@ public class TestHMasterRPCException {
       }
       fail();
     } finally {
-      engine.close();
+      rpcClient.stop();
     }
   }
 }
\ No newline at end of file

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java?rev=1478639&r1=1478638&r2=1478639&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java Fri May  3 03:58:33 2013
@@ -132,6 +132,4 @@ public class TestRestartCluster {
       UTIL.waitTableAvailable(TABLE);
     }
   }
-
-}
-
+}
\ No newline at end of file

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java?rev=1478639&r1=1478638&r2=1478639&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java Fri May  3 03:58:33 2013
@@ -30,7 +30,6 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.*;
 import org.apache.hadoop.hbase.MultithreadedTestUtil.RepeatingTestThread;
 import org.apache.hadoop.hbase.MultithreadedTestUtil.TestContext;
-import org.apache.hadoop.hbase.client.AdminProtocol;
 import org.apache.hadoop.hbase.client.HConnection;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Result;
@@ -42,6 +41,7 @@ import org.apache.hadoop.hbase.io.compre
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.protobuf.RequestConverter;
+import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionRequest;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequest;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -152,7 +152,7 @@ public class TestHRegionServerBulkLoad {
           byte[] regionName = location.getRegionInfo().getRegionName();
           BulkLoadHFileRequest request =
             RequestConverter.buildBulkLoadHFileRequest(famPaths, regionName, true);
-          server.bulkLoadHFile(null, request);
+          stub.bulkLoadHFile(null, request);
           return null;
         }
       }.withRetries();
@@ -166,7 +166,8 @@ public class TestHRegionServerBulkLoad {
           public Void call() throws Exception {
             LOG.debug("compacting " + location + " for row "
                 + Bytes.toStringBinary(row));
-            AdminProtocol server = connection.getAdmin(location.getServerName());
+            AdminProtos.AdminService.BlockingInterface server =
+              connection.getAdmin(location.getServerName());
             CompactRegionRequest request =
               RequestConverter.buildCompactRegionRequest(
                 location.getRegionInfo().getRegionName(), true, null);

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogWriter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogWriter.java?rev=1478639&r1=1478638&r2=1478639&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogWriter.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogWriter.java Fri May  3 03:58:33 2013
@@ -148,11 +148,11 @@ public class SequenceFileLogWriter imple
         null,
         createMetadata(conf, compress));
     } else {
-      LOG.debug("using new createWriter -- HADOOP-6840");
+      if (LOG.isTraceEnabled()) LOG.trace("Using new createWriter -- HADOOP-6840");
     }
     
     this.writer_out = getSequenceFilePrivateFSDataOutputStreamAccessible();
-    LOG.debug("Path=" + path + ", compression=" + compress);
+    if (LOG.isTraceEnabled()) LOG.trace("Path=" + path + ", compression=" + compress);
   }
 
   // Get at the private FSDataOutputStream inside in SequenceFile so we can

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java?rev=1478639&r1=1478638&r2=1478639&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java Fri May  3 03:58:33 2013
@@ -18,21 +18,25 @@
 
 package org.apache.hadoop.hbase.security.token;
 
+import static org.apache.hadoop.fs.CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+import java.util.List;
 import java.util.concurrent.ConcurrentMap;
 
+import com.google.protobuf.BlockingRpcChannel;
+import com.google.protobuf.BlockingService;
 import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.ClusterId;
-import org.apache.hadoop.hbase.IpcProtocol;
 import org.apache.hadoop.hbase.Coprocessor;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
@@ -43,18 +47,16 @@ import org.apache.hadoop.hbase.catalog.C
 import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.ipc.BlockingRpcCallback;
-import org.apache.hadoop.hbase.ipc.HBaseClientRPC;
-import org.apache.hadoop.hbase.ipc.HBaseServer;
-import org.apache.hadoop.hbase.ipc.HBaseServerRPC;
-import org.apache.hadoop.hbase.ipc.ProtobufRpcClientEngine;
-import org.apache.hadoop.hbase.ipc.RequestContext;
+import org.apache.hadoop.hbase.ipc.RpcClient;
 import org.apache.hadoop.hbase.ipc.RpcServer;
+import org.apache.hadoop.hbase.ipc.RequestContext;
+import org.apache.hadoop.hbase.ipc.RpcServer.BlockingServiceAndInterface;
+import org.apache.hadoop.hbase.ipc.RpcServerInterface;
 import org.apache.hadoop.hbase.ipc.ServerRpcController;
 import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
-import org.apache.hadoop.hbase.security.KerberosInfo;
-import org.apache.hadoop.hbase.security.TokenInfo;
+import org.apache.hadoop.hbase.security.SecurityInfo;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
@@ -67,6 +69,8 @@ import org.apache.hadoop.hbase.zookeeper
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.hadoop.net.DNS;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authorize.PolicyProvider;
+import org.apache.hadoop.security.authorize.Service;
 import org.apache.hadoop.security.token.SecretManager;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
@@ -82,23 +86,16 @@ import org.junit.experimental.categories
 public class TestTokenAuthentication {
   private static Log LOG = LogFactory.getLog(TestTokenAuthentication.class);
 
-  @KerberosInfo(
-      serverPrincipal = "hbase.test.kerberos.principal")
-  @TokenInfo("HBASE_AUTH_TOKEN")
-  private static interface BlockingAuthenticationService
-  extends AuthenticationProtos.AuthenticationService.BlockingInterface, IpcProtocol {
-  }
+  public static interface AuthenticationServiceSecurityInfo {}
 
   /**
    * Basic server process for RPC authentication testing
    */
   private static class TokenServer extends TokenProvider
-      implements BlockingAuthenticationService, Runnable, Server {
-
+  implements AuthenticationProtos.AuthenticationService.BlockingInterface, Runnable, Server {
     private static Log LOG = LogFactory.getLog(TokenServer.class);
-
     private Configuration conf;
-    private RpcServer rpcServer;
+    private RpcServerInterface rpcServer;
     private InetSocketAddress isa;
     private ZooKeeperWatcher zookeeper;
     private Sleeper sleeper;
@@ -106,31 +103,27 @@ public class TestTokenAuthentication {
     private boolean aborted = false;
     private boolean stopped = false;
     private long startcode;
-    private AuthenticationProtos.AuthenticationService.BlockingInterface blockingService;
 
     public TokenServer(Configuration conf) throws IOException {
       this.conf = conf;
       this.startcode = EnvironmentEdgeManager.currentTimeMillis();
-
       // Server to handle client requests.
-      String hostname = Strings.domainNamePointerToHostName(
-          DNS.getDefaultHost("default", "default"));
+      String hostname =
+        Strings.domainNamePointerToHostName(DNS.getDefaultHost("default", "default"));
       int port = 0;
       // Creation of an ISA will force a resolve.
       InetSocketAddress initialIsa = new InetSocketAddress(hostname, port);
       if (initialIsa.getAddress() == null) {
         throw new IllegalArgumentException("Failed resolve of " + initialIsa);
       }
-
-      this.rpcServer = HBaseServerRPC.getServer(TokenServer.class, this,
-          new Class<?>[]{AuthenticationProtos.AuthenticationService.Interface.class},
-          initialIsa.getHostName(), // BindAddress is IP we got for this server.
-          initialIsa.getPort(),
-          3, // handlers
-          1, // meta handlers (not used)
-          true,
-          this.conf, HConstants.QOS_THRESHOLD);
-      // Set our address.
+      final List<BlockingServiceAndInterface> sai =
+        new ArrayList<BlockingServiceAndInterface>(1);
+      BlockingService service =
+        AuthenticationProtos.AuthenticationService.newReflectiveBlockingService(this);
+      sai.add(new BlockingServiceAndInterface(service,
+        AuthenticationProtos.AuthenticationService.BlockingInterface.class));
+      this.rpcServer =
+        new RpcServer(this, "tokenServer", sai, initialIsa, 3, 1, conf, HConstants.QOS_THRESHOLD);
       this.isa = this.rpcServer.getListenerAddress();
       this.sleeper = new Sleeper(1000, this);
     }
@@ -179,7 +172,7 @@ public class TestTokenAuthentication {
       // mock RegionServerServices to provide to coprocessor environment
       final RegionServerServices mockServices = new MockRegionServerServices() {
         @Override
-        public RpcServer getRpcServer() { return rpcServer; }
+        public RpcServerInterface getRpcServer() { return rpcServer; }
       };
 
       // mock up coprocessor environment
@@ -253,7 +246,7 @@ public class TestTokenAuthentication {
     }
 
     public SecretManager<? extends TokenIdentifier> getSecretManager() {
-      return ((HBaseServer)rpcServer).getSecretManager();
+      return ((RpcServer)rpcServer).getSecretManager();
     }
 
     @Override
@@ -304,19 +297,30 @@ public class TestTokenAuthentication {
   public static void setupBeforeClass() throws Exception {
     TEST_UTIL = new HBaseTestingUtility();
     TEST_UTIL.startMiniZKCluster();
+    // register token type for protocol
+    SecurityInfo.addInfo(AuthenticationProtos.AuthenticationService.getDescriptor().getName(),
+      new SecurityInfo("hbase.test.kerberos.principal",
+        AuthenticationProtos.TokenIdentifier.Kind.HBASE_AUTH_TOKEN));
     // security settings only added after startup so that ZK does not require SASL
     Configuration conf = TEST_UTIL.getConfiguration();
     conf.set("hadoop.security.authentication", "kerberos");
     conf.set("hbase.security.authentication", "kerberos");
+    conf.setBoolean(HADOOP_SECURITY_AUTHORIZATION, true);
     server = new TokenServer(conf);
     serverThread = new Thread(server);
-    Threads.setDaemonThreadRunning(serverThread,
-        "TokenServer:"+server.getServerName().toString());
+    Threads.setDaemonThreadRunning(serverThread, "TokenServer:"+server.getServerName().toString());
     // wait for startup
     while (!server.isStarted() && !server.isStopped()) {
       Thread.sleep(10);
     }
-
+    server.rpcServer.refreshAuthManager(new PolicyProvider() {
+      @Override
+      public Service[] getServices() {
+        return new Service [] {
+          new Service("security.client.protocol.acl",
+            AuthenticationProtos.AuthenticationService.BlockingInterface.class)};
+      }
+    });
     ZKClusterId.setClusterId(server.getZooKeeper(), clusterId);
     secretManager = (AuthenticationTokenSecretManager)server.getSecretManager();
     while(secretManager.getCurrentKey() == null) {
@@ -363,24 +367,23 @@ public class TestTokenAuthentication {
     testuser.doAs(new PrivilegedExceptionAction<Object>() {
       public Object run() throws Exception {
         Configuration c = server.getConfiguration();
-        ProtobufRpcClientEngine rpcClient =
-            new ProtobufRpcClientEngine(c, clusterId.toString());
+        RpcClient rpcClient = new RpcClient(c, clusterId.toString());
+        ServerName sn =
+          new ServerName(server.getAddress().getHostName(), server.getAddress().getPort(),
+            System.currentTimeMillis());
         try {
-          AuthenticationProtos.AuthenticationService.BlockingInterface proxy =
-              HBaseClientRPC.waitForProxy(rpcClient, BlockingAuthenticationService.class,
-                  server.getAddress(), c,
-                  HConstants.DEFAULT_HBASE_CLIENT_RPC_MAXATTEMPTS,
-                  HConstants.DEFAULT_HBASE_RPC_TIMEOUT,
-                  HConstants.DEFAULT_HBASE_RPC_TIMEOUT);
-
+          BlockingRpcChannel channel = rpcClient.createBlockingRpcChannel(sn,
+            User.getCurrent(), HConstants.DEFAULT_HBASE_RPC_TIMEOUT);
+          AuthenticationProtos.AuthenticationService.BlockingInterface stub =
+            AuthenticationProtos.AuthenticationService.newBlockingStub(channel);
           AuthenticationProtos.WhoAmIResponse response =
-              proxy.whoami(null, AuthenticationProtos.WhoAmIRequest.getDefaultInstance());
+            stub.whoami(null, AuthenticationProtos.WhoAmIRequest.getDefaultInstance());
           String myname = response.getUsername();
           assertEquals("testuser", myname);
           String authMethod = response.getAuthMethod();
           assertEquals("TOKEN", authMethod);
         } finally {
-          rpcClient.close();
+          rpcClient.stop();
         }
         return null;
       }

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java?rev=1478639&r1=1478638&r2=1478639&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java Fri May  3 03:58:33 2013
@@ -40,8 +40,8 @@ import org.apache.hadoop.hbase.HTableDes
 import org.apache.hadoop.hbase.LargeTests;
 import org.apache.hadoop.hbase.exceptions.SnapshotCreationException;
 import org.apache.hadoop.hbase.exceptions.TableNotFoundException;
-import org.apache.hadoop.hbase.ipc.HBaseClient;
-import org.apache.hadoop.hbase.ipc.HBaseServer;
+import org.apache.hadoop.hbase.ipc.RpcClient;
+import org.apache.hadoop.hbase.ipc.RpcServer;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.ScannerCallable;
@@ -87,8 +87,8 @@ public class TestFlushSnapshotFromClient
    */
   @BeforeClass
   public static void setupCluster() throws Exception {
-    ((Log4JLogger)HBaseServer.LOG).getLogger().setLevel(Level.ALL);
-    ((Log4JLogger)HBaseClient.LOG).getLogger().setLevel(Level.ALL);
+    ((Log4JLogger)RpcServer.LOG).getLogger().setLevel(Level.ALL);
+    ((Log4JLogger)RpcClient.LOG).getLogger().setLevel(Level.ALL);
     ((Log4JLogger)ScannerCallable.LOG).getLogger().setLevel(Level.ALL);
     setupConf(UTIL.getConfiguration());
     UTIL.startMiniCluster(NUM_RS);

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreSnapshotHelper.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreSnapshotHelper.java?rev=1478639&r1=1478638&r2=1478639&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreSnapshotHelper.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreSnapshotHelper.java Fri May  3 03:58:33 2013
@@ -19,10 +19,8 @@ package org.apache.hadoop.hbase.snapshot
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
 
 import java.io.IOException;
-import java.util.Arrays;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
@@ -35,25 +33,23 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 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.HTableDescriptor;
-import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.SmallTests;
 import org.apache.hadoop.hbase.catalog.CatalogTracker;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HConnectionTestingUtility;
 import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
 import org.apache.hadoop.hbase.io.HFileLink;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
 import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
-import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSTableDescriptors;
 import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.hadoop.hbase.util.MD5Hash;
-import org.junit.*;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.mockito.Mockito;
 

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockRegionServerServices.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockRegionServerServices.java?rev=1478639&r1=1478638&r2=1478639&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockRegionServerServices.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockRegionServerServices.java Fri May  3 03:58:33 2013
@@ -30,7 +30,7 @@ import org.apache.hadoop.hbase.ServerNam
 import org.apache.hadoop.hbase.catalog.CatalogTracker;
 import org.apache.hadoop.hbase.executor.ExecutorService;
 import org.apache.hadoop.hbase.fs.HFileSystem;
-import org.apache.hadoop.hbase.ipc.RpcServer;
+import org.apache.hadoop.hbase.ipc.RpcServerInterface;
 import org.apache.hadoop.hbase.master.TableLockManager;
 import org.apache.hadoop.hbase.master.TableLockManager.NullTableLockManager;
 import org.apache.hadoop.hbase.regionserver.CompactionRequestor;
@@ -99,7 +99,7 @@ public class MockRegionServerServices im
   }
 
   @Override
-  public RpcServer getRpcServer() {
+  public RpcServerInterface getRpcServer() {
     return null;
   }
 

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java?rev=1478639&r1=1478638&r2=1478639&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java Fri May  3 03:58:33 2013
@@ -59,7 +59,6 @@ import org.apache.hadoop.hbase.LargeTest
 import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.catalog.MetaEditor;
-import org.apache.hadoop.hbase.client.AdminProtocol;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
@@ -77,6 +76,7 @@ import org.apache.hadoop.hbase.master.Re
 import org.apache.hadoop.hbase.master.TableLockManager;
 import org.apache.hadoop.hbase.master.TableLockManager.TableLock;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
@@ -524,8 +524,7 @@ public class TestHBaseFsck {
         new HashMap<ServerName, List<String>>();
     HConnection connection = admin.getConnection();
     for (ServerName hsi : regionServers) {
-      AdminProtocol server =
-        connection.getAdmin(hsi);
+      AdminProtos.AdminService.BlockingInterface server = connection.getAdmin(hsi);
 
       // list all online regions from this region server
       List<HRegionInfo> regions = ProtobufUtil.getOnlineRegions(server);

Modified: hbase/branches/0.95/hbase-server/src/test/protobuf/test_delayed_rpc.proto
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/protobuf/test_delayed_rpc.proto?rev=1478639&r1=1478638&r2=1478639&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/protobuf/test_delayed_rpc.proto (original)
+++ hbase/branches/0.95/hbase-server/src/test/protobuf/test_delayed_rpc.proto Fri May  3 03:58:33 2013
@@ -28,3 +28,7 @@ message TestArg {
 message TestResponse {
   required int32 response = 1;
 }
+
+service TestDelayedService {
+  rpc test(TestArg) returns (TestResponse);
+}