You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zh...@apache.org on 2016/09/08 12:33:17 UTC

[1/6] hbase git commit: HBASE-16445 Refactor and reimplement RpcClient

Repository: hbase
Updated Branches:
  refs/heads/master fc224ed0e -> c04b38918


http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcClientLeaks.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcClientLeaks.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcClientLeaks.java
index d3e5767..e4ecd10 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcClientLeaks.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcClientLeaks.java
@@ -20,6 +20,8 @@ package org.apache.hadoop.hbase.ipc;
 import static org.apache.hadoop.hbase.HBaseTestingUtility.fam1;
 import static org.junit.Assert.assertTrue;
 
+import com.google.common.collect.Lists;
+
 import java.io.IOException;
 import java.net.Socket;
 import java.net.SocketAddress;
@@ -38,9 +40,7 @@ import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.MetricsConnection;
 import org.apache.hadoop.hbase.client.RetriesExhaustedException;
 import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.codec.Codec;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.io.compress.CompressionCodec;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Rule;
@@ -49,19 +49,17 @@ import org.junit.experimental.categories.Category;
 import org.junit.rules.ExpectedException;
 import org.junit.rules.TestRule;
 
-import com.google.common.collect.Lists;
-
 @Category(MediumTests.class)
 public class TestRpcClientLeaks {
   @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()).
       withLookingForStuckThread(true).build();
 
-  public static class MyRpcClientImpl extends RpcClientImpl {
+  public static class MyRpcClientImpl extends BlockingRpcClient {
     public static List<Socket> savedSockets = Lists.newArrayList();
     @Rule public ExpectedException thrown = ExpectedException.none();
 
-    public MyRpcClientImpl(Configuration conf, String clusterId) {
-      super(conf, clusterId);
+    public MyRpcClientImpl(Configuration conf) {
+      super(conf);
     }
 
     public MyRpcClientImpl(Configuration conf, String clusterId, SocketAddress address,
@@ -70,9 +68,8 @@ public class TestRpcClientLeaks {
     }
 
     @Override
-    protected Connection createConnection(ConnectionId remoteId, Codec codec,
-        CompressionCodec compressor) throws IOException {
-      return new Connection(remoteId, codec, compressor) {
+    protected BlockingRpcConnection createConnection(ConnectionId remoteId) throws IOException {
+      return new BlockingRpcConnection(this, remoteId) {
         @Override
         protected synchronized void setupConnection() throws IOException {
           super.setupConnection();

http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcHandlerException.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcHandlerException.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcHandlerException.java
index 749009f..3bb85e9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcHandlerException.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcHandlerException.java
@@ -29,7 +29,6 @@ import java.net.InetSocketAddress;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoRequestProto;
 import org.apache.hadoop.hbase.ipc.protobuf.generated.TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface;
 import org.apache.hadoop.hbase.testclassification.RPCTests;
@@ -86,7 +85,7 @@ public class TestRpcHandlerException {
     Abortable abortable = new AbortServer();
     RpcScheduler scheduler = new SimpleRpcScheduler(CONF, 2, 0, 0, qosFunction, abortable, 0);
     RpcServer rpcServer = new TestRpcServer(scheduler);
-    try (RpcClientImpl client = new RpcClientImpl(CONF, HConstants.CLUSTER_ID_DEFAULT)) {
+    try (BlockingRpcClient client = new BlockingRpcClient(CONF)) {
       rpcServer.start();
       BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress());
       stub.echo(null, EchoRequestProto.newBuilder().setMessage("hello").build());

http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicaFailover.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicaFailover.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicaFailover.java
index b3b04c2..2496e2b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicaFailover.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicaFailover.java
@@ -190,6 +190,7 @@ public class TestRegionReplicaFailover {
             LOG.info("Aborting region server hosting primary region replica");
             rs.getRegionServer().abort("for test");
             aborted = true;
+            break;
           }
         }
       }
@@ -251,6 +252,7 @@ public class TestRegionReplicaFailover {
             LOG.info("Aborting region server hosting secondary region replica");
             rs.getRegionServer().abort("for test");
             aborted = true;
+            break;
           }
         }
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-server/src/test/java/org/apache/hadoop/hbase/security/AbstractTestSecureIPC.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/AbstractTestSecureIPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/AbstractTestSecureIPC.java
deleted file mode 100644
index c1b8de7..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/AbstractTestSecureIPC.java
+++ /dev/null
@@ -1,261 +0,0 @@
-/**
- *
- * 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.security;
-
-import static org.apache.hadoop.hbase.ipc.TestProtobufRpcServiceImpl.SERVICE;
-import static org.apache.hadoop.hbase.ipc.TestProtobufRpcServiceImpl.newBlockingStub;
-import static org.apache.hadoop.hbase.security.HBaseKerberosUtils.getKeytabFileForTesting;
-import static org.apache.hadoop.hbase.security.HBaseKerberosUtils.getPrincipalForTesting;
-import static org.apache.hadoop.hbase.security.HBaseKerberosUtils.getSecuredConfiguration;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotSame;
-import static org.junit.Assert.assertSame;
-
-import com.google.common.collect.Lists;
-import com.google.protobuf.ServiceException;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Properties;
-
-import javax.security.sasl.SaslException;
-
-import org.apache.commons.lang.RandomStringUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.ipc.FifoRpcScheduler;
-import org.apache.hadoop.hbase.ipc.RpcClient;
-import org.apache.hadoop.hbase.ipc.RpcClientFactory;
-import org.apache.hadoop.hbase.ipc.RpcServer;
-import org.apache.hadoop.hbase.ipc.RpcServerInterface;
-import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos;
-import org.apache.hadoop.hbase.ipc.protobuf.generated.TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface;
-import org.apache.hadoop.minikdc.MiniKdc;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.mockito.Mockito;
-
-public abstract class AbstractTestSecureIPC {
-
-  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-
-  private static final File KEYTAB_FILE = new File(TEST_UTIL.getDataTestDir("keytab").toUri()
-      .getPath());
-
-  private static MiniKdc KDC;
-  private static String HOST = "localhost";
-  private static String PRINCIPAL;
-
-  String krbKeytab;
-  String krbPrincipal;
-  UserGroupInformation ugi;
-  Configuration clientConf;
-  Configuration serverConf;
-
-  abstract Class<? extends RpcClient> getRpcClientClass();
-
-  @Rule
-  public ExpectedException exception = ExpectedException.none();
-
-  @BeforeClass
-  public static void setUp() throws Exception {
-    Properties conf = MiniKdc.createConf();
-    conf.put(MiniKdc.DEBUG, true);
-    KDC = new MiniKdc(conf, new File(TEST_UTIL.getDataTestDir("kdc").toUri().getPath()));
-    KDC.start();
-    PRINCIPAL = "hbase/" + HOST;
-    KDC.createPrincipal(KEYTAB_FILE, PRINCIPAL);
-    HBaseKerberosUtils.setKeytabFileForTesting(KEYTAB_FILE.getAbsolutePath());
-    HBaseKerberosUtils.setPrincipalForTesting(PRINCIPAL + "@" + KDC.getRealm());
-  }
-
-  @AfterClass
-  public static void tearDown() throws IOException {
-    if (KDC != null) {
-      KDC.stop();
-    }
-    TEST_UTIL.cleanupTestDir();
-  }
-
-  @Before
-  public void setUpTest() throws Exception {
-    krbKeytab = getKeytabFileForTesting();
-    krbPrincipal = getPrincipalForTesting();
-    ugi = loginKerberosPrincipal(krbKeytab, krbPrincipal);
-    clientConf = getSecuredConfiguration();
-    clientConf.set(RpcClientFactory.CUSTOM_RPC_CLIENT_IMPL_CONF_KEY, getRpcClientClass().getName());
-    serverConf = getSecuredConfiguration();
-  }
-
-  @Test
-  public void testRpcCallWithEnabledKerberosSaslAuth() throws Exception {
-    UserGroupInformation ugi2 = UserGroupInformation.getCurrentUser();
-
-    // check that the login user is okay:
-    assertSame(ugi, ugi2);
-    assertEquals(AuthenticationMethod.KERBEROS, ugi.getAuthenticationMethod());
-    assertEquals(krbPrincipal, ugi.getUserName());
-
-    callRpcService(User.create(ugi2));
-  }
-
-  @Test
-  public void testRpcFallbackToSimpleAuth() throws Exception {
-    String clientUsername = "testuser";
-    UserGroupInformation clientUgi = UserGroupInformation.createUserForTesting(clientUsername,
-        new String[]{clientUsername});
-
-    // check that the client user is insecure
-    assertNotSame(ugi, clientUgi);
-    assertEquals(AuthenticationMethod.SIMPLE, clientUgi.getAuthenticationMethod());
-    assertEquals(clientUsername, clientUgi.getUserName());
-
-    clientConf.set(User.HBASE_SECURITY_CONF_KEY, "simple");
-    serverConf.setBoolean(RpcServer.FALLBACK_TO_INSECURE_CLIENT_AUTH, true);
-    callRpcService(User.create(clientUgi));
-  }
-
-  void setRpcProtection(String clientProtection, String serverProtection) {
-    clientConf.set("hbase.rpc.protection", clientProtection);
-    serverConf.set("hbase.rpc.protection", serverProtection);
-  }
-
-  /**
-   * Test various combinations of Server and Client qops.
-   * @throws Exception
-   */
-  @Test
-  public void testSaslWithCommonQop() throws Exception {
-    setRpcProtection("privacy,authentication", "authentication");
-    callRpcService(User.create(ugi));
-
-    setRpcProtection("authentication", "privacy,authentication");
-    callRpcService(User.create(ugi));
-
-    setRpcProtection("integrity,authentication", "privacy,authentication");
-    callRpcService(User.create(ugi));
-
-    setRpcProtection("integrity,authentication", "integrity,authentication");
-    callRpcService(User.create(ugi));
-
-    setRpcProtection("privacy,authentication", "privacy,authentication");
-    callRpcService(User.create(ugi));
-  }
-
-  @Test
-  public void testSaslNoCommonQop() throws Exception {
-    exception.expect(SaslException.class);
-    exception.expectMessage("No common protection layer between client and server");
-    setRpcProtection("integrity", "privacy");
-    callRpcService(User.create(ugi));
-  }
-
-  private UserGroupInformation loginKerberosPrincipal(String krbKeytab, String krbPrincipal)
-      throws Exception {
-    Configuration cnf = new Configuration();
-    cnf.set(CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION, "kerberos");
-    UserGroupInformation.setConfiguration(cnf);
-    UserGroupInformation.loginUserFromKeytab(krbPrincipal, krbKeytab);
-    return UserGroupInformation.getLoginUser();
-  }
-
-  /**
-   * Sets up a RPC Server and a Client. Does a RPC checks the result. If an exception is thrown
-   * from the stub, this function will throw root cause of that exception.
-   */
-  private void callRpcService(User clientUser) throws Exception {
-    SecurityInfo securityInfoMock = Mockito.mock(SecurityInfo.class);
-    Mockito.when(securityInfoMock.getServerPrincipal())
-        .thenReturn(HBaseKerberosUtils.KRB_PRINCIPAL);
-    SecurityInfo.addInfo("TestProtobufRpcProto", securityInfoMock);
-
-    InetSocketAddress isa = new InetSocketAddress(HOST, 0);
-
-    RpcServerInterface rpcServer =
-        new RpcServer(null, "AbstractTestSecureIPC",
-            Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(SERVICE, null)), isa,
-            serverConf, new FifoRpcScheduler(serverConf, 1));
-    rpcServer.start();
-    try (RpcClient rpcClient = RpcClientFactory.createClient(clientConf,
-        HConstants.DEFAULT_CLUSTER_ID.toString())) {
-      BlockingInterface stub = newBlockingStub(rpcClient, rpcServer.getListenerAddress(),
-        clientUser);
-      List<String> results = new ArrayList<>();
-      TestThread th1 = new TestThread(stub, results);
-      final Throwable exception[] = new Throwable[1];
-      Collections.synchronizedList(new ArrayList<Throwable>());
-      Thread.UncaughtExceptionHandler exceptionHandler =
-          new Thread.UncaughtExceptionHandler() {
-            public void uncaughtException(Thread th, Throwable ex) {
-              exception[0] = ex;
-            }
-          };
-      th1.setUncaughtExceptionHandler(exceptionHandler);
-      th1.start();
-      th1.join();
-      if (exception[0] != null) {
-        // throw root cause.
-        while (exception[0].getCause() != null) {
-          exception[0] = exception[0].getCause();
-        }
-        throw (Exception) exception[0];
-      }
-    } finally {
-      rpcServer.stop();
-    }
-  }
-
-  public static class TestThread extends Thread {
-      private final BlockingInterface stub;
-
-      private final List<String> results;
-
-          public TestThread(BlockingInterface stub, List<String> results) {
-          this.stub = stub;
-          this.results = results;
-        }
-
-          @Override
-      public void run() {
-          try {
-            int[] messageSize = new int[] {100, 1000, 10000};
-            for (int i = 0; i < messageSize.length; i++) {
-              String input = RandomStringUtils.random(messageSize[i]);
-              String result = stub.echo(null, TestProtos.EchoRequestProto.newBuilder()
-                  .setMessage(input).build()).getMessage();
-              assertEquals(input, result);
-            }
-          } catch (ServiceException e) {
-            throw new RuntimeException(e);
-          }
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestAsyncSecureIPC.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestAsyncSecureIPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestAsyncSecureIPC.java
deleted file mode 100644
index ea37915..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestAsyncSecureIPC.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/**
- *
- * 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.security;
-
-import org.apache.hadoop.hbase.ipc.AsyncRpcClient;
-import org.apache.hadoop.hbase.ipc.RpcClient;
-import org.apache.hadoop.hbase.testclassification.SecurityTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.junit.experimental.categories.Category;
-
-@Category({ SecurityTests.class, SmallTests.class })
-public class TestAsyncSecureIPC extends AbstractTestSecureIPC {
-
-  Class<? extends RpcClient> getRpcClientClass() {
-    return AsyncRpcClient.class;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestSecureIPC.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestSecureIPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestSecureIPC.java
index 98ea221..7ddefad 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestSecureIPC.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestSecureIPC.java
@@ -18,16 +18,258 @@
  */
 package org.apache.hadoop.hbase.security;
 
+import static org.apache.hadoop.hbase.ipc.TestProtobufRpcServiceImpl.SERVICE;
+import static org.apache.hadoop.hbase.ipc.TestProtobufRpcServiceImpl.newBlockingStub;
+import static org.apache.hadoop.hbase.security.HBaseKerberosUtils.getKeytabFileForTesting;
+import static org.apache.hadoop.hbase.security.HBaseKerberosUtils.getPrincipalForTesting;
+import static org.apache.hadoop.hbase.security.HBaseKerberosUtils.getSecuredConfiguration;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotSame;
+import static org.junit.Assert.assertSame;
+
+import com.google.common.collect.Lists;
+import com.google.protobuf.ServiceException;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Properties;
+
+import javax.security.sasl.SaslException;
+
+import org.apache.commons.lang.RandomStringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.ipc.BlockingRpcClient;
+import org.apache.hadoop.hbase.ipc.FifoRpcScheduler;
+import org.apache.hadoop.hbase.ipc.NettyRpcClient;
 import org.apache.hadoop.hbase.ipc.RpcClient;
-import org.apache.hadoop.hbase.ipc.RpcClientImpl;
+import org.apache.hadoop.hbase.ipc.RpcClientFactory;
+import org.apache.hadoop.hbase.ipc.RpcServer;
+import org.apache.hadoop.hbase.ipc.RpcServerInterface;
+import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos;
+import org.apache.hadoop.hbase.ipc.protobuf.generated.TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface;
 import org.apache.hadoop.hbase.testclassification.SecurityTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.minikdc.MiniKdc;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameter;
+import org.junit.runners.Parameterized.Parameters;
+import org.mockito.Mockito;
 
+@RunWith(Parameterized.class)
 @Category({ SecurityTests.class, SmallTests.class })
-public class TestSecureIPC extends AbstractTestSecureIPC {
+public class TestSecureIPC {
+
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+
+  private static final File KEYTAB_FILE = new File(
+      TEST_UTIL.getDataTestDir("keytab").toUri().getPath());
+
+  private static MiniKdc KDC;
+  private static String HOST = "localhost";
+  private static String PRINCIPAL;
+
+  String krbKeytab;
+  String krbPrincipal;
+  UserGroupInformation ugi;
+  Configuration clientConf;
+  Configuration serverConf;
+
+  @Rule
+  public ExpectedException exception = ExpectedException.none();
+
+  @Parameters(name = "{index}: rpcClientImpl={0}")
+  public static Collection<Object[]> parameters() {
+    return Arrays.asList(new Object[] { BlockingRpcClient.class.getName() },
+      new Object[] { NettyRpcClient.class.getName() });
+  }
+
+  @Parameter
+  public String rpcClientImpl;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    Properties conf = MiniKdc.createConf();
+    conf.put(MiniKdc.DEBUG, true);
+    KDC = new MiniKdc(conf, new File(TEST_UTIL.getDataTestDir("kdc").toUri().getPath()));
+    KDC.start();
+    PRINCIPAL = "hbase/" + HOST;
+    KDC.createPrincipal(KEYTAB_FILE, PRINCIPAL);
+    HBaseKerberosUtils.setKeytabFileForTesting(KEYTAB_FILE.getAbsolutePath());
+    HBaseKerberosUtils.setPrincipalForTesting(PRINCIPAL + "@" + KDC.getRealm());
+  }
+
+  @AfterClass
+  public static void tearDown() throws IOException {
+    if (KDC != null) {
+      KDC.stop();
+    }
+    TEST_UTIL.cleanupTestDir();
+  }
+
+  @Before
+  public void setUpTest() throws Exception {
+    krbKeytab = getKeytabFileForTesting();
+    krbPrincipal = getPrincipalForTesting();
+    ugi = loginKerberosPrincipal(krbKeytab, krbPrincipal);
+    clientConf = getSecuredConfiguration();
+    clientConf.set(RpcClientFactory.CUSTOM_RPC_CLIENT_IMPL_CONF_KEY, rpcClientImpl);
+    serverConf = getSecuredConfiguration();
+  }
+
+  @Test
+  public void testRpcCallWithEnabledKerberosSaslAuth() throws Exception {
+    UserGroupInformation ugi2 = UserGroupInformation.getCurrentUser();
+
+    // check that the login user is okay:
+    assertSame(ugi, ugi2);
+    assertEquals(AuthenticationMethod.KERBEROS, ugi.getAuthenticationMethod());
+    assertEquals(krbPrincipal, ugi.getUserName());
+
+    callRpcService(User.create(ugi2));
+  }
+
+  @Test
+  public void testRpcFallbackToSimpleAuth() throws Exception {
+    String clientUsername = "testuser";
+    UserGroupInformation clientUgi = UserGroupInformation.createUserForTesting(clientUsername,
+      new String[] { clientUsername });
+
+    // check that the client user is insecure
+    assertNotSame(ugi, clientUgi);
+    assertEquals(AuthenticationMethod.SIMPLE, clientUgi.getAuthenticationMethod());
+    assertEquals(clientUsername, clientUgi.getUserName());
+
+    clientConf.set(User.HBASE_SECURITY_CONF_KEY, "simple");
+    serverConf.setBoolean(RpcServer.FALLBACK_TO_INSECURE_CLIENT_AUTH, true);
+    callRpcService(User.create(clientUgi));
+  }
+
+  void setRpcProtection(String clientProtection, String serverProtection) {
+    clientConf.set("hbase.rpc.protection", clientProtection);
+    serverConf.set("hbase.rpc.protection", serverProtection);
+  }
+
+  /**
+   * Test various combinations of Server and Client qops.
+   * @throws Exception
+   */
+  @Test
+  public void testSaslWithCommonQop() throws Exception {
+    setRpcProtection("privacy,authentication", "authentication");
+    callRpcService(User.create(ugi));
+
+    setRpcProtection("authentication", "privacy,authentication");
+    callRpcService(User.create(ugi));
+
+    setRpcProtection("integrity,authentication", "privacy,authentication");
+    callRpcService(User.create(ugi));
+
+    setRpcProtection("integrity,authentication", "integrity,authentication");
+    callRpcService(User.create(ugi));
+
+    setRpcProtection("privacy,authentication", "privacy,authentication");
+    callRpcService(User.create(ugi));
+  }
+
+  @Test
+  public void testSaslNoCommonQop() throws Exception {
+    exception.expect(SaslException.class);
+    exception.expectMessage("No common protection layer between client and server");
+    setRpcProtection("integrity", "privacy");
+    callRpcService(User.create(ugi));
+  }
+
+  private UserGroupInformation loginKerberosPrincipal(String krbKeytab, String krbPrincipal)
+      throws Exception {
+    Configuration cnf = new Configuration();
+    cnf.set(CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION, "kerberos");
+    UserGroupInformation.setConfiguration(cnf);
+    UserGroupInformation.loginUserFromKeytab(krbPrincipal, krbKeytab);
+    return UserGroupInformation.getLoginUser();
+  }
+
+  /**
+   * Sets up a RPC Server and a Client. Does a RPC checks the result. If an exception is thrown from
+   * the stub, this function will throw root cause of that exception.
+   */
+  private void callRpcService(User clientUser) throws Exception {
+    SecurityInfo securityInfoMock = Mockito.mock(SecurityInfo.class);
+    Mockito.when(securityInfoMock.getServerPrincipal())
+        .thenReturn(HBaseKerberosUtils.KRB_PRINCIPAL);
+    SecurityInfo.addInfo("TestProtobufRpcProto", securityInfoMock);
+
+    InetSocketAddress isa = new InetSocketAddress(HOST, 0);
+
+    RpcServerInterface rpcServer = new RpcServer(null, "AbstractTestSecureIPC",
+        Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(SERVICE, null)), isa,
+        serverConf, new FifoRpcScheduler(serverConf, 1));
+    rpcServer.start();
+    try (RpcClient rpcClient = RpcClientFactory.createClient(clientConf,
+      HConstants.DEFAULT_CLUSTER_ID.toString())) {
+      BlockingInterface stub = newBlockingStub(rpcClient, rpcServer.getListenerAddress(),
+        clientUser);
+      TestThread th1 = new TestThread(stub);
+      final Throwable exception[] = new Throwable[1];
+      Collections.synchronizedList(new ArrayList<Throwable>());
+      Thread.UncaughtExceptionHandler exceptionHandler = new Thread.UncaughtExceptionHandler() {
+        public void uncaughtException(Thread th, Throwable ex) {
+          exception[0] = ex;
+        }
+      };
+      th1.setUncaughtExceptionHandler(exceptionHandler);
+      th1.start();
+      th1.join();
+      if (exception[0] != null) {
+        // throw root cause.
+        while (exception[0].getCause() != null) {
+          exception[0] = exception[0].getCause();
+        }
+        throw (Exception) exception[0];
+      }
+    } finally {
+      rpcServer.stop();
+    }
+  }
+
+  public static class TestThread extends Thread {
+    private final BlockingInterface stub;
+
+    public TestThread(BlockingInterface stub) {
+      this.stub = stub;
+    }
 
-  Class<? extends RpcClient> getRpcClientClass() {
-    return RpcClientImpl.class;
+    @Override
+    public void run() {
+      try {
+        int[] messageSize = new int[] { 100, 1000, 10000 };
+        for (int i = 0; i < messageSize.length; i++) {
+          String input = RandomStringUtils.random(messageSize[i]);
+          String result = stub
+              .echo(null, TestProtos.EchoRequestProto.newBuilder().setMessage(input).build())
+              .getMessage();
+          assertEquals(input, result);
+        }
+      } catch (ServiceException e) {
+        throw new RuntimeException(e);
+      }
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestDelegationTokenWithEncryption.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestDelegationTokenWithEncryption.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestDelegationTokenWithEncryption.java
index 60d4a6a..aa8c7ee 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestDelegationTokenWithEncryption.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestDelegationTokenWithEncryption.java
@@ -19,6 +19,9 @@
 package org.apache.hadoop.hbase.security.token;
 
 import com.google.protobuf.ServiceException;
+
+import java.io.IOException;
+
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
@@ -29,10 +32,10 @@ import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.ipc.AsyncRpcClient;
+import org.apache.hadoop.hbase.ipc.BlockingRpcClient;
+import org.apache.hadoop.hbase.ipc.NettyRpcClient;
 import org.apache.hadoop.hbase.ipc.RpcClient;
 import org.apache.hadoop.hbase.ipc.RpcClientFactory;
-import org.apache.hadoop.hbase.ipc.RpcClientImpl;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.SecurityTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -43,7 +46,6 @@ import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-import java.io.IOException;
 
 @Category({ SecurityTests.class, MediumTests.class })
 public class TestDelegationTokenWithEncryption extends SecureTestCluster {
@@ -84,8 +86,8 @@ public class TestDelegationTokenWithEncryption extends SecureTestCluster {
       tableDescriptor.addFamily(new HColumnDescriptor("family"));
       admin.createTable(tableDescriptor);
 
-      testPutGetWithDelegationToken(RpcClientImpl.class);
-      testPutGetWithDelegationToken(AsyncRpcClient.class);
+      testPutGetWithDelegationToken(BlockingRpcClient.class);
+      testPutGetWithDelegationToken(NettyRpcClient.class);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestGenerateDelegationToken.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestGenerateDelegationToken.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestGenerateDelegationToken.java
index 1d7c676..d403867 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestGenerateDelegationToken.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestGenerateDelegationToken.java
@@ -27,11 +27,11 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.ipc.AsyncRpcClient;
+import org.apache.hadoop.hbase.ipc.NettyRpcClient;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
 import org.apache.hadoop.hbase.ipc.RpcClient;
 import org.apache.hadoop.hbase.ipc.RpcClientFactory;
-import org.apache.hadoop.hbase.ipc.RpcClientImpl;
+import org.apache.hadoop.hbase.ipc.BlockingRpcClient;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos;
 import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos.GetAuthenticationTokenRequest;
@@ -79,8 +79,8 @@ public class TestGenerateDelegationToken extends SecureTestCluster {
     try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration())) {
       Token<? extends TokenIdentifier> token = TokenUtil.obtainToken(conn);
       UserGroupInformation.getCurrentUser().addToken(token);
-      testTokenAuth(RpcClientImpl.class);
-      testTokenAuth(AsyncRpcClient.class);
+      testTokenAuth(BlockingRpcClient.class);
+      testTokenAuth(NettyRpcClient.class);
     }
   }
 }


[4/6] hbase git commit: HBASE-16445 Refactor and reimplement RpcClient

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java
index c18bd7e..f66350b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java
@@ -30,15 +30,19 @@ import java.nio.ByteBuffer;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.exceptions.ConnectionClosingException;
+import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta;
 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse;
+import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader;
+import org.apache.hadoop.hbase.protobuf.generated.TracingProtos.RPCTInfo;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.ipc.RemoteException;
 
 /**
  * Utility to help ipc'ing.
  */
 @InterfaceAudience.Private
-public class IPCUtil {
+class IPCUtil {
 
   /**
    * Write out header, param, and cell block if there is one.
@@ -93,18 +97,32 @@ public class IPCUtil {
     return totalSize;
   }
 
-  /**
-   * @return True if the exception is a fatal connection exception.
-   */
-  public static boolean isFatalConnectionException(final ExceptionResponse e) {
-    return e.getExceptionClassName().equals(FatalConnectionException.class.getName());
+  static RequestHeader buildRequestHeader(Call call, CellBlockMeta cellBlockMeta) {
+    RequestHeader.Builder builder = RequestHeader.newBuilder();
+    builder.setCallId(call.id);
+    if (call.span != null) {
+      builder.setTraceInfo(RPCTInfo.newBuilder().setParentId(call.span.getSpanId())
+          .setTraceId(call.span.getTraceId()));
+    }
+    builder.setMethodName(call.md.getName());
+    builder.setRequestParam(call.param != null);
+    if (cellBlockMeta != null) {
+      builder.setCellBlockMeta(cellBlockMeta);
+    }
+    // Only pass priority if there is one set.
+    if (call.priority != HBaseRpcController.PRIORITY_UNSET) {
+      builder.setPriority(call.priority);
+    }
+    builder.setTimeout(call.timeout);
+
+    return builder.build();
   }
 
   /**
    * @param e exception to be wrapped
    * @return RemoteException made from passed <code>e</code>
    */
-  public static RemoteException createRemoteException(final ExceptionResponse e) {
+  static RemoteException createRemoteException(final ExceptionResponse e) {
     String innerExceptionClassName = e.getExceptionClassName();
     boolean doNotRetry = e.getDoNotRetry();
     return e.hasHostname() ?
@@ -115,6 +133,21 @@ public class IPCUtil {
   }
 
   /**
+   * @return True if the exception is a fatal connection exception.
+   */
+  static boolean isFatalConnectionException(final ExceptionResponse e) {
+    return e.getExceptionClassName().equals(FatalConnectionException.class.getName());
+  }
+
+  static IOException toIOE(Throwable t) {
+    if (t instanceof IOException) {
+      return (IOException) t;
+    } else {
+      return new IOException(t);
+    }
+  }
+
+  /**
    * Takes an Exception and the address we were trying to connect to and return an IOException with
    * the input exception as the cause. The new exception provides the stack trace of the place where
    * the exception is thrown and some extra diagnostics information. If the exception is
@@ -124,7 +157,7 @@ public class IPCUtil {
    * @param exception the relevant exception
    * @return an exception to throw
    */
-  public static IOException wrapException(InetSocketAddress addr, Exception exception) {
+  static IOException wrapException(InetSocketAddress addr, Exception exception) {
     if (exception instanceof ConnectException) {
       // connection refused; include the host:port in the error
       return (ConnectException) new ConnectException(
@@ -140,4 +173,10 @@ public class IPCUtil {
           "Call to " + addr + " failed on local exception: " + exception).initCause(exception);
     }
   }
+
+  static void setCancelled(Call call) {
+    call.setException(new CallCancelledException("Call id=" + call.id + ", waitTime="
+        + (EnvironmentEdgeManager.currentTime() - call.getStartTime()) + ", rpcTimetout="
+        + call.timeout));
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/MessageConverter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/MessageConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/MessageConverter.java
deleted file mode 100644
index a85225a..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/MessageConverter.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/**
- * 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.ipc;
-
-import com.google.protobuf.Message;
-import java.io.IOException;
-import org.apache.hadoop.hbase.CellScanner;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-
-/**
- * Interface to convert Messages to specific types
- * @param <M> Message Type to convert
- * @param <O> Output Type
- */
-@InterfaceAudience.Private
-public interface MessageConverter<M,O> {
-  /**
-   * Converts Message to Output
-   * @param msg to convert
-   * @param cellScanner to use for conversion
-   * @return Output
-   * @throws IOException if message could not be converted to response
-   */
-  O convert(M msg, CellScanner cellScanner) throws IOException;
-
-  MessageConverter<Message,Message> NO_CONVERTER = new MessageConverter<Message, Message>() {
-    @Override
-    public Message convert(Message msg, CellScanner cellScanner) throws IOException {
-      return msg;
-    }
-  };
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClient.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClient.java
new file mode 100644
index 0000000..8c568af
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClient.java
@@ -0,0 +1,80 @@
+/**
+ * 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.ipc;
+
+import io.netty.channel.Channel;
+import io.netty.channel.EventLoopGroup;
+import io.netty.channel.nio.NioEventLoopGroup;
+import io.netty.channel.socket.nio.NioSocketChannel;
+
+import java.io.IOException;
+import java.net.SocketAddress;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.MetricsConnection;
+import org.apache.hadoop.hbase.util.Pair;
+
+/**
+ * Netty client for the requests and responses.
+ */
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
+public class NettyRpcClient extends AbstractRpcClient<NettyRpcConnection> {
+
+  final EventLoopGroup group;
+
+  final Class<? extends Channel> channelClass;
+
+  private final boolean shutdownGroupWhenClose;
+
+  public NettyRpcClient(Configuration configuration, String clusterId, SocketAddress localAddress,
+      MetricsConnection metrics) {
+    super(configuration, clusterId, localAddress, metrics);
+    Pair<EventLoopGroup, Class<? extends Channel>> groupAndChannelClass = NettyRpcClientConfigHelper
+        .getEventLoopConfig(conf);
+    if (groupAndChannelClass == null) {
+      // Use our own EventLoopGroup.
+      this.group = new NioEventLoopGroup();
+      this.channelClass = NioSocketChannel.class;
+      this.shutdownGroupWhenClose = true;
+    } else {
+      this.group = groupAndChannelClass.getFirst();
+      this.channelClass = groupAndChannelClass.getSecond();
+      this.shutdownGroupWhenClose = false;
+    }
+  }
+
+  /** Used in test only. */
+  NettyRpcClient(Configuration configuration) {
+    this(configuration, HConstants.CLUSTER_ID_DEFAULT, null, null);
+  }
+
+  @Override
+  protected NettyRpcConnection createConnection(ConnectionId remoteId) throws IOException {
+    return new NettyRpcConnection(this, remoteId);
+  }
+
+  @Override
+  protected void closeInternal() {
+    if (shutdownGroupWhenClose) {
+      group.shutdownGracefully();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClientConfigHelper.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClientConfigHelper.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClientConfigHelper.java
new file mode 100644
index 0000000..a8af69c
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClientConfigHelper.java
@@ -0,0 +1,83 @@
+/**
+ * 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.ipc;
+
+import com.google.common.base.Preconditions;
+
+import io.netty.channel.Channel;
+import io.netty.channel.EventLoopGroup;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.util.Pair;
+
+/**
+ * Helper class for passing config to {@link NettyRpcClient}.
+ * <p>
+ * As hadoop Configuration can not pass an Object directly, we need to find a way to pass the
+ * EventLoopGroup to {@code AsyncRpcClient} if we want to use a single {@code EventLoopGroup} for
+ * the whole process.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class NettyRpcClientConfigHelper {
+
+  public static final String EVENT_LOOP_CONFIG = "hbase.rpc.client.event-loop.config";
+
+  private static final String CONFIG_NAME = "global-event-loop";
+
+  private static final Map<String, Pair<EventLoopGroup, Class<? extends Channel>>>
+    EVENT_LOOP_CONFIG_MAP = new HashMap<>();
+
+  /**
+   * Set the EventLoopGroup and channel class for {@code AsyncRpcClient}.
+   */
+  public static void setEventLoopConfig(Configuration conf, EventLoopGroup group,
+      Class<? extends Channel> channelClass) {
+    Preconditions.checkNotNull(group, "group is null");
+    Preconditions.checkNotNull(channelClass, "channel class is null");
+    conf.set(EVENT_LOOP_CONFIG, CONFIG_NAME);
+    EVENT_LOOP_CONFIG_MAP.put(CONFIG_NAME,
+      Pair.<EventLoopGroup, Class<? extends Channel>> newPair(group, channelClass));
+  }
+
+  /**
+   * The {@code AsyncRpcClient} will create its own {@code NioEventLoopGroup}.
+   */
+  public static void createEventLoopPerClient(Configuration conf) {
+    conf.set(EVENT_LOOP_CONFIG, "");
+    EVENT_LOOP_CONFIG_MAP.clear();
+  }
+
+  static Pair<EventLoopGroup, Class<? extends Channel>> getEventLoopConfig(Configuration conf) {
+    String name = conf.get(EVENT_LOOP_CONFIG);
+    if (name == null) {
+      return DefaultNettyEventLoopConfig.GROUP_AND_CHANNEL_CLASS;
+    }
+    if (StringUtils.isBlank(name)) {
+      return null;
+    }
+    return EVENT_LOOP_CONFIG_MAP.get(name);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcConnection.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcConnection.java
new file mode 100644
index 0000000..5f22dfd
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcConnection.java
@@ -0,0 +1,282 @@
+/**
+ * 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.ipc;
+
+import static org.apache.hadoop.hbase.ipc.CallEvent.Type.CANCELLED;
+import static org.apache.hadoop.hbase.ipc.CallEvent.Type.TIMEOUT;
+import static org.apache.hadoop.hbase.ipc.IPCUtil.setCancelled;
+import static org.apache.hadoop.hbase.ipc.IPCUtil.toIOE;
+
+import com.google.protobuf.RpcCallback;
+
+import io.netty.bootstrap.Bootstrap;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufOutputStream;
+import io.netty.buffer.Unpooled;
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelFutureListener;
+import io.netty.channel.ChannelHandler;
+import io.netty.channel.ChannelOption;
+import io.netty.channel.ChannelPipeline;
+import io.netty.handler.codec.LengthFieldBasedFrameDecoder;
+import io.netty.handler.timeout.IdleStateHandler;
+import io.netty.util.concurrent.Future;
+import io.netty.util.concurrent.FutureListener;
+import io.netty.util.concurrent.Promise;
+
+import java.io.IOException;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.ipc.BufferCallBeforeInitHandler.BufferCallEvent;
+import org.apache.hadoop.hbase.ipc.HBaseRpcController.CancellationCallback;
+import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ConnectionHeader;
+import org.apache.hadoop.hbase.security.AsyncHBaseSaslRpcClientHandler;
+import org.apache.hadoop.hbase.security.SaslChallengeDecoder;
+import org.apache.hadoop.hbase.security.SaslUtil.QualityOfProtection;
+import org.apache.hadoop.hbase.util.Threads;
+import org.apache.hadoop.security.UserGroupInformation;
+
+/**
+ * RPC connection implementation based on netty.
+ * <p>
+ * Most operations are executed in handlers. Netty handler is always executed in the same
+ * thread(EventLoop) so no lock is needed.
+ */
+@InterfaceAudience.Private
+class NettyRpcConnection extends RpcConnection {
+
+  private static final Log LOG = LogFactory.getLog(NettyRpcConnection.class);
+
+  private static final ScheduledExecutorService RELOGIN_EXECUTOR = Executors
+      .newSingleThreadScheduledExecutor(Threads.newDaemonThreadFactory("Relogin"));
+
+  private final NettyRpcClient rpcClient;
+
+  private ByteBuf connectionHeaderPreamble;
+
+  private ByteBuf connectionHeaderWithLength;
+
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "IS2_INCONSISTENT_SYNC",
+      justification = "connect is also under lock as notifyOnCancel will call our action directly")
+  private Channel channel;
+
+  NettyRpcConnection(NettyRpcClient rpcClient, ConnectionId remoteId) throws IOException {
+    super(rpcClient.conf, AbstractRpcClient.WHEEL_TIMER, remoteId, rpcClient.clusterId,
+        rpcClient.userProvider.isHBaseSecurityEnabled(), rpcClient.codec, rpcClient.compressor);
+    this.rpcClient = rpcClient;
+    byte[] connectionHeaderPreamble = getConnectionHeaderPreamble();
+    this.connectionHeaderPreamble = Unpooled.directBuffer(connectionHeaderPreamble.length)
+        .writeBytes(connectionHeaderPreamble);
+    ConnectionHeader header = getConnectionHeader();
+    this.connectionHeaderWithLength = Unpooled.directBuffer(4 + header.getSerializedSize());
+    this.connectionHeaderWithLength.writeInt(header.getSerializedSize());
+    header.writeTo(new ByteBufOutputStream(this.connectionHeaderWithLength));
+  }
+
+  @Override
+  protected synchronized void callTimeout(Call call) {
+    if (channel != null) {
+      channel.pipeline().fireUserEventTriggered(new CallEvent(TIMEOUT, call));
+    }
+  }
+
+  @Override
+  public synchronized boolean isActive() {
+    return channel != null;
+  }
+
+  private void shutdown0() {
+    if (channel != null) {
+      channel.close();
+      channel = null;
+    }
+  }
+
+  @Override
+  public synchronized void shutdown() {
+    shutdown0();
+  }
+
+  private void established(Channel ch) {
+    ch.write(connectionHeaderWithLength.retainedDuplicate());
+    ChannelPipeline p = ch.pipeline();
+    String addBeforeHandler = p.context(BufferCallBeforeInitHandler.class).name();
+    p.addBefore(addBeforeHandler, null,
+      new IdleStateHandler(0, rpcClient.minIdleTimeBeforeClose, 0, TimeUnit.MILLISECONDS));
+    p.addBefore(addBeforeHandler, null, new LengthFieldBasedFrameDecoder(Integer.MAX_VALUE, 0, 4));
+    p.addBefore(addBeforeHandler, null,
+      new NettyRpcDuplexHandler(this, rpcClient.cellBlockBuilder, codec, compressor));
+    p.fireUserEventTriggered(BufferCallEvent.success());
+  }
+
+  private boolean reloginInProgress;
+
+  private void scheduleRelogin(Throwable error) {
+    if (error instanceof FallbackDisallowedException) {
+      return;
+    }
+    synchronized (this) {
+      if (reloginInProgress) {
+        return;
+      }
+      reloginInProgress = true;
+      RELOGIN_EXECUTOR.schedule(new Runnable() {
+
+        @Override
+        public void run() {
+          try {
+            if (shouldAuthenticateOverKrb()) {
+              relogin();
+            }
+          } catch (IOException e) {
+            LOG.warn("relogin failed", e);
+          }
+          synchronized (this) {
+            reloginInProgress = false;
+          }
+        }
+      }, ThreadLocalRandom.current().nextInt(reloginMaxBackoff), TimeUnit.MILLISECONDS);
+    }
+  }
+
+  private void failInit(Channel ch, IOException e) {
+    synchronized (this) {
+      // fail all pending calls
+      ch.pipeline().fireUserEventTriggered(BufferCallEvent.fail(e));
+      shutdown0();
+      return;
+    }
+  }
+
+  private void saslNegotiate(final Channel ch) {
+    UserGroupInformation ticket = getUGI();
+    if (ticket == null) {
+      failInit(ch, new FatalConnectionException("ticket/user is null"));
+      return;
+    }
+    Promise<Boolean> saslPromise = ch.eventLoop().newPromise();
+    ChannelHandler saslHandler;
+    try {
+      saslHandler = new AsyncHBaseSaslRpcClientHandler(saslPromise, ticket, authMethod, token,
+          serverPrincipal, rpcClient.fallbackAllowed, this.rpcClient.conf.get(
+            "hbase.rpc.protection", QualityOfProtection.AUTHENTICATION.name().toLowerCase()));
+    } catch (IOException e) {
+      failInit(ch, e);
+      return;
+    }
+    ch.pipeline().addFirst(new SaslChallengeDecoder(), saslHandler);
+    saslPromise.addListener(new FutureListener<Boolean>() {
+
+      @Override
+      public void operationComplete(Future<Boolean> future) throws Exception {
+        if (future.isSuccess()) {
+          ChannelPipeline p = ch.pipeline();
+          p.remove(SaslChallengeDecoder.class);
+          p.remove(AsyncHBaseSaslRpcClientHandler.class);
+          established(ch);
+        } else {
+          final Throwable error = future.cause();
+          scheduleRelogin(error);
+          failInit(ch, toIOE(error));
+        }
+      }
+    });
+  }
+
+  private void connect() {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Connecting to " + remoteId.address);
+    }
+
+    this.channel = new Bootstrap().group(rpcClient.group).channel(rpcClient.channelClass)
+        .option(ChannelOption.TCP_NODELAY, rpcClient.isTcpNoDelay())
+        .option(ChannelOption.SO_KEEPALIVE, rpcClient.tcpKeepAlive)
+        .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, rpcClient.connectTO)
+        .handler(new BufferCallBeforeInitHandler()).localAddress(rpcClient.localAddr)
+        .remoteAddress(remoteId.address).connect().addListener(new ChannelFutureListener() {
+
+          @Override
+          public void operationComplete(ChannelFuture future) throws Exception {
+            Channel ch = future.channel();
+            if (!future.isSuccess()) {
+              failInit(ch, toIOE(future.cause()));
+              rpcClient.failedServers.addToFailedServers(remoteId.address);
+              return;
+            }
+            ch.writeAndFlush(connectionHeaderPreamble.retainedDuplicate());
+            if (useSasl) {
+              saslNegotiate(ch);
+            } else {
+              established(ch);
+            }
+          }
+        }).channel();
+  }
+
+  @Override
+  public synchronized void sendRequest(final Call call, HBaseRpcController pcrc)
+      throws IOException {
+    if (reloginInProgress) {
+      throw new IOException("Can not send request because relogin is in progress.");
+    }
+    pcrc.notifyOnCancel(new RpcCallback<Object>() {
+
+      @Override
+      public void run(Object parameter) {
+        setCancelled(call);
+        synchronized (this) {
+          if (channel != null) {
+            channel.pipeline().fireUserEventTriggered(new CallEvent(CANCELLED, call));
+          }
+        }
+      }
+    }, new CancellationCallback() {
+
+      @Override
+      public void run(boolean cancelled) throws IOException {
+        if (cancelled) {
+          setCancelled(call);
+        } else {
+          if (channel == null) {
+            connect();
+          }
+          scheduleTimeoutTask(call);
+          channel.writeAndFlush(call).addListener(new ChannelFutureListener() {
+
+            @Override
+            public void operationComplete(ChannelFuture future) throws Exception {
+              // Fail the call if we failed to write it out. This usually because the channel is
+              // closed. This is needed because we may shutdown the channel inside event loop and
+              // there may still be some pending calls in the event loop queue after us.
+              if (!future.isSuccess()) {
+                call.setException(toIOE(future.cause()));
+              }
+            }
+          });
+        }
+      }
+    });
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcDuplexHandler.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcDuplexHandler.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcDuplexHandler.java
new file mode 100644
index 0000000..1cd89d8
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcDuplexHandler.java
@@ -0,0 +1,245 @@
+/**
+ * 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.ipc;
+
+import com.google.protobuf.Message;
+import com.google.protobuf.Message.Builder;
+import com.google.protobuf.TextFormat;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufInputStream;
+import io.netty.buffer.ByteBufOutputStream;
+import io.netty.channel.ChannelDuplexHandler;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelPromise;
+import io.netty.handler.timeout.IdleStateEvent;
+import io.netty.util.concurrent.PromiseCombiner;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.CellScanner;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.codec.Codec;
+import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta;
+import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse;
+import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader;
+import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.ipc.RemoteException;
+
+/**
+ * The netty rpc handler.
+ */
+@InterfaceAudience.Private
+class NettyRpcDuplexHandler extends ChannelDuplexHandler {
+
+  private static final Log LOG = LogFactory.getLog(NettyRpcDuplexHandler.class);
+
+  private final NettyRpcConnection conn;
+
+  private final CellBlockBuilder cellBlockBuilder;
+
+  private final Codec codec;
+
+  private final CompressionCodec compressor;
+
+  private final Map<Integer, Call> id2Call = new HashMap<Integer, Call>();
+
+  public NettyRpcDuplexHandler(NettyRpcConnection conn, CellBlockBuilder cellBlockBuilder,
+      Codec codec, CompressionCodec compressor) {
+    this.conn = conn;
+    this.cellBlockBuilder = cellBlockBuilder;
+    this.codec = codec;
+    this.compressor = compressor;
+
+  }
+
+  private void writeRequest(ChannelHandlerContext ctx, Call call, ChannelPromise promise)
+      throws IOException {
+    id2Call.put(call.id, call);
+    ByteBuf cellBlock = cellBlockBuilder.buildCellBlock(codec, compressor, call.cells, ctx.alloc());
+    CellBlockMeta cellBlockMeta;
+    if (cellBlock != null) {
+      CellBlockMeta.Builder cellBlockMetaBuilder = CellBlockMeta.newBuilder();
+      cellBlockMetaBuilder.setLength(cellBlock.writerIndex());
+      cellBlockMeta = cellBlockMetaBuilder.build();
+    } else {
+      cellBlockMeta = null;
+    }
+    RequestHeader requestHeader = IPCUtil.buildRequestHeader(call, cellBlockMeta);
+    int sizeWithoutCellBlock = IPCUtil.getTotalSizeWhenWrittenDelimited(requestHeader, call.param);
+    int totalSize = cellBlock != null ? sizeWithoutCellBlock + cellBlock.writerIndex()
+        : sizeWithoutCellBlock;
+    ByteBuf buf = ctx.alloc().buffer(sizeWithoutCellBlock + 4);
+    buf.writeInt(totalSize);
+    ByteBufOutputStream bbos = new ByteBufOutputStream(buf);
+    requestHeader.writeDelimitedTo(bbos);
+    if (call.param != null) {
+      call.param.writeDelimitedTo(bbos);
+    }
+    if (cellBlock != null) {
+      ChannelPromise withoutCellBlockPromise = ctx.newPromise();
+      ctx.write(buf, withoutCellBlockPromise);
+      ChannelPromise cellBlockPromise = ctx.newPromise();
+      ctx.write(cellBlock, cellBlockPromise);
+      PromiseCombiner combiner = new PromiseCombiner();
+      combiner.addAll(withoutCellBlockPromise, cellBlockPromise);
+      combiner.finish(promise);
+    } else {
+      ctx.write(buf, promise);
+    }
+  }
+
+  @Override
+  public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise)
+      throws Exception {
+    if (msg instanceof Call) {
+      writeRequest(ctx, (Call) msg, promise);
+    } else {
+      ctx.write(msg, promise);
+    }
+  }
+
+  private void readResponse(ChannelHandlerContext ctx, ByteBuf buf) throws IOException {
+    int totalSize = buf.readInt();
+    ByteBufInputStream in = new ByteBufInputStream(buf);
+    ResponseHeader responseHeader = ResponseHeader.parseDelimitedFrom(in);
+    int id = responseHeader.getCallId();
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("got response header " + TextFormat.shortDebugString(responseHeader)
+          + ", totalSize: " + totalSize + " bytes");
+    }
+    RemoteException remoteExc;
+    if (responseHeader.hasException()) {
+      ExceptionResponse exceptionResponse = responseHeader.getException();
+      remoteExc = IPCUtil.createRemoteException(exceptionResponse);
+      if (IPCUtil.isFatalConnectionException(exceptionResponse)) {
+        // Here we will cleanup all calls so do not need to fall back, just return.
+        exceptionCaught(ctx, remoteExc);
+        return;
+      }
+    } else {
+      remoteExc = null;
+    }
+    Call call = id2Call.remove(id);
+    if (call == null) {
+      // So we got a response for which we have no corresponding 'call' here on the client-side.
+      // We probably timed out waiting, cleaned up all references, and now the server decides
+      // to return a response. There is nothing we can do w/ the response at this stage. Clean
+      // out the wire of the response so its out of the way and we can get other responses on
+      // this connection.
+      int readSoFar = IPCUtil.getTotalSizeWhenWrittenDelimited(responseHeader);
+      int whatIsLeftToRead = totalSize - readSoFar;
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Unknown callId: " + id + ", skipping over this response of " + whatIsLeftToRead
+            + " bytes");
+      }
+      return;
+    }
+    if (remoteExc != null) {
+      call.setException(remoteExc);
+      return;
+    }
+    Message value;
+    if (call.responseDefaultType != null) {
+      Builder builder = call.responseDefaultType.newBuilderForType();
+      builder.mergeDelimitedFrom(in);
+      value = builder.build();
+    } else {
+      value = null;
+    }
+    CellScanner cellBlockScanner;
+    if (responseHeader.hasCellBlockMeta()) {
+      int size = responseHeader.getCellBlockMeta().getLength();
+      // Maybe we could read directly from the ByteBuf.
+      // The problem here is that we do not know when to release it.
+      byte[] cellBlock = new byte[size];
+      buf.readBytes(cellBlock);
+      cellBlockScanner = cellBlockBuilder.createCellScanner(this.codec, this.compressor, cellBlock);
+    } else {
+      cellBlockScanner = null;
+    }
+    call.setResponse(value, cellBlockScanner);
+  }
+
+  @Override
+  public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
+    if (msg instanceof ByteBuf) {
+      ByteBuf buf = (ByteBuf) msg;
+      try {
+        readResponse(ctx, buf);
+      } finally {
+        buf.release();
+      }
+    } else {
+      super.channelRead(ctx, msg);
+    }
+  }
+
+  private void cleanupCalls(ChannelHandlerContext ctx, IOException error) {
+    for (Call call : id2Call.values()) {
+      call.setException(error);
+    }
+    id2Call.clear();
+  }
+
+  @Override
+  public void channelInactive(ChannelHandlerContext ctx) throws Exception {
+    cleanupCalls(ctx, new IOException("Connection closed"));
+    conn.shutdown();
+  }
+
+  @Override
+  public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) {
+    cleanupCalls(ctx, IPCUtil.toIOE(cause));
+    conn.shutdown();
+  }
+
+  @Override
+  public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exception {
+    if (evt instanceof IdleStateEvent) {
+      IdleStateEvent idleEvt = (IdleStateEvent) evt;
+      switch (idleEvt.state()) {
+        case WRITER_IDLE:
+          if (id2Call.isEmpty()) {
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("shutdown connection to " + conn.remoteId().address
+                  + " because idle for a long time");
+            }
+            // It may happen that there are still some pending calls in the event loop queue and
+            // they will get a closed channel exception. But this is not a big deal as it rarely
+            // rarely happens and the upper layer could retry immediately.
+            conn.shutdown();
+          }
+          break;
+        default:
+          LOG.warn("Unrecognized idle state " + idleEvt.state());
+          break;
+      }
+    } else if (evt instanceof CallEvent) {
+      // just remove the call for now until we add other call event other than timeout and cancel.
+      id2Call.remove(((CallEvent) evt).call.id);
+    } else {
+      ctx.fireUserEventTriggered(evt);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java
index a8ec628..26a5739 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java
@@ -30,7 +30,8 @@ import org.apache.hadoop.hbase.security.User;
 /**
  * Interface for RpcClient implementations so ConnectionManager can handle it.
  */
-@InterfaceAudience.Private public interface RpcClient extends Closeable {
+@InterfaceAudience.Private
+public interface RpcClient extends Closeable {
   String FAILED_SERVER_EXPIRY_KEY = "hbase.ipc.client.failed.servers.expiry";
   int FAILED_SERVER_EXPIRY_DEFAULT = 2000;
   String IDLE_TIME = "hbase.ipc.client.connection.minIdleTimeBeforeClose";
@@ -79,7 +80,8 @@ import org.apache.hadoop.hbase.security.User;
    *
    * @return A rpc channel that goes via this rpc client instance.
    */
-  RpcChannel createProtobufRpcChannel(final ServerName sn, final User user, int rpcTimeout);
+  RpcChannel createRpcChannel(final ServerName sn, final User user, int rpcTimeout)
+      throws IOException;
 
   /**
    * Interrupt the connections to the given server. This should be called if the server

http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientFactory.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientFactory.java
index 07254e9..8cdfb03 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientFactory.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientFactory.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hbase.ipc;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.ImmutableMap;
 
 import java.net.SocketAddress;
 
@@ -34,6 +35,10 @@ public final class RpcClientFactory {
 
   public static final String CUSTOM_RPC_CLIENT_IMPL_CONF_KEY = "hbase.rpc.client.impl";
 
+  private static final ImmutableMap<String, String> DEPRECATED_NAME_MAPPING = ImmutableMap.of(
+    "org.apache.hadoop.hbase.ipc.RpcClientImpl", BlockingRpcClient.class.getName(),
+    "org.apache.hadoop.hbase.ipc.AsyncRpcClient", NettyRpcClient.class.getName());
+
   /**
    * Private Constructor
    */
@@ -59,6 +64,15 @@ public final class RpcClientFactory {
     return createClient(conf, clusterId, null, metrics);
   }
 
+  private static String getRpcClientClass(Configuration conf) {
+    String rpcClientClass = conf.get(CUSTOM_RPC_CLIENT_IMPL_CONF_KEY);
+    if (rpcClientClass == null) {
+      return NettyRpcClient.class.getName();
+    }
+    String mappedName = DEPRECATED_NAME_MAPPING.get(rpcClientClass);
+    return mappedName == null ? rpcClientClass : mappedName;
+  }
+
   /**
    * Creates a new RpcClient by the class defined in the configuration or falls back to
    * RpcClientImpl
@@ -70,13 +84,9 @@ public final class RpcClientFactory {
    */
   public static RpcClient createClient(Configuration conf, String clusterId,
       SocketAddress localAddr, MetricsConnection metrics) {
-    String rpcClientClass =
-        conf.get(CUSTOM_RPC_CLIENT_IMPL_CONF_KEY, AsyncRpcClient.class.getName());
-    return ReflectionUtils.instantiateWithCustomCtor(
-        rpcClientClass,
-        new Class[] { Configuration.class, String.class, SocketAddress.class,
-          MetricsConnection.class },
-        new Object[] { conf, clusterId, localAddr, metrics }
-    );
+    String rpcClientClass = getRpcClientClass(conf);
+    return ReflectionUtils.instantiateWithCustomCtor(rpcClientClass, new Class[] {
+        Configuration.class, String.class, SocketAddress.class, MetricsConnection.class },
+      new Object[] { conf, clusterId, localAddr, metrics });
   }
 }
\ No newline at end of file


[6/6] hbase git commit: HBASE-16445 Refactor and reimplement RpcClient

Posted by zh...@apache.org.
HBASE-16445 Refactor and reimplement RpcClient


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/c04b3891
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/c04b3891
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/c04b3891

Branch: refs/heads/master
Commit: c04b389181b6a9299f05f1ad8f8b1ec62448331a
Parents: fc224ed
Author: zhangduo <zh...@apache.org>
Authored: Thu Sep 8 17:46:33 2016 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu Sep 8 20:32:56 2016 +0800

----------------------------------------------------------------------
 .../RpcRetryingCallerWithReadReplicas.java      |   21 +-
 .../hadoop/hbase/ipc/AbstractRpcClient.java     |  467 ++++--
 .../org/apache/hadoop/hbase/ipc/AsyncCall.java  |  204 ---
 .../hadoop/hbase/ipc/AsyncRpcChannel.java       |  768 ----------
 .../apache/hadoop/hbase/ipc/AsyncRpcClient.java |  510 -------
 .../hbase/ipc/AsyncServerResponseHandler.java   |  126 --
 .../hadoop/hbase/ipc/BlockingRpcCallback.java   |    2 +-
 .../hadoop/hbase/ipc/BlockingRpcClient.java     |   77 +
 .../hadoop/hbase/ipc/BlockingRpcConnection.java |  725 ++++++++++
 .../hbase/ipc/BufferCallBeforeInitHandler.java  |  103 ++
 .../java/org/apache/hadoop/hbase/ipc/Call.java  |  122 +-
 .../hbase/ipc/CallCancelledException.java       |   37 +
 .../org/apache/hadoop/hbase/ipc/CallEvent.java  |   40 +
 .../hadoop/hbase/ipc/CallTimeoutException.java  |    6 +-
 .../hadoop/hbase/ipc/CellBlockBuilder.java      |  111 +-
 .../apache/hadoop/hbase/ipc/ConnectionId.java   |    2 +-
 .../hbase/ipc/DefaultNettyEventLoopConfig.java  |   37 +
 .../hbase/ipc/FallbackDisallowedException.java  |   38 +
 .../hadoop/hbase/ipc/IOExceptionConverter.java  |   34 -
 .../org/apache/hadoop/hbase/ipc/IPCUtil.java    |   55 +-
 .../hadoop/hbase/ipc/MessageConverter.java      |   47 -
 .../apache/hadoop/hbase/ipc/NettyRpcClient.java |   80 ++
 .../hbase/ipc/NettyRpcClientConfigHelper.java   |   83 ++
 .../hadoop/hbase/ipc/NettyRpcConnection.java    |  282 ++++
 .../hadoop/hbase/ipc/NettyRpcDuplexHandler.java |  245 ++++
 .../org/apache/hadoop/hbase/ipc/RpcClient.java  |    6 +-
 .../hadoop/hbase/ipc/RpcClientFactory.java      |   26 +-
 .../apache/hadoop/hbase/ipc/RpcClientImpl.java  | 1359 ------------------
 .../apache/hadoop/hbase/ipc/RpcConnection.java  |  255 ++++
 .../security/AbstractHBaseSaslRpcClient.java    |  197 +++
 .../hbase/security/AsyncHBaseSaslRpcClient.java |   58 +
 .../AsyncHBaseSaslRpcClientHandler.java         |  135 ++
 .../hbase/security/HBaseSaslRpcClient.java      |  234 +--
 .../hbase/security/SaslChallengeDecoder.java    |  112 ++
 .../hbase/security/SaslClientHandler.java       |  382 -----
 .../hbase/security/SaslUnwrapHandler.java       |   53 +
 .../apache/hadoop/hbase/security/SaslUtil.java  |   14 +-
 .../hadoop/hbase/security/SaslWrapHandler.java  |   80 ++
 .../hadoop/hbase/ipc/TestCellBlockBuilder.java  |   19 +-
 .../ipc/TestRpcClientDeprecatedNameMapping.java |   56 +
 .../hbase/security/TestHBaseSaslRpcClient.java  |    8 +-
 .../hbase/ipc/IntegrationTestRpcClient.java     |   25 +-
 .../org/apache/hadoop/hbase/ipc/RpcServer.java  |    2 +-
 .../hadoop/hbase/master/ServerManager.java      |    1 +
 .../hadoop/hbase/util/MultiHConnection.java     |    1 -
 .../hadoop/hbase/client/TestClientTimeouts.java |   11 +-
 .../hbase/client/TestRpcControllerFactory.java  |    4 +-
 .../hadoop/hbase/ipc/AbstractTestIPC.java       |  121 +-
 .../apache/hadoop/hbase/ipc/TestAsyncIPC.java   |  113 --
 .../hadoop/hbase/ipc/TestBlockingIPC.java       |   58 +
 .../hbase/ipc/TestGlobalEventLoopGroup.java     |   53 -
 .../org/apache/hadoop/hbase/ipc/TestIPC.java    |   74 -
 .../apache/hadoop/hbase/ipc/TestNettyIPC.java   |  128 ++
 .../hbase/ipc/TestProtobufRpcServiceImpl.java   |    2 +-
 .../hadoop/hbase/ipc/TestRpcClientLeaks.java    |   17 +-
 .../hbase/ipc/TestRpcHandlerException.java      |    3 +-
 .../regionserver/TestRegionReplicaFailover.java |    2 +
 .../hbase/security/AbstractTestSecureIPC.java   |  261 ----
 .../hbase/security/TestAsyncSecureIPC.java      |   33 -
 .../hadoop/hbase/security/TestSecureIPC.java    |  250 +++-
 .../TestDelegationTokenWithEncryption.java      |   12 +-
 .../token/TestGenerateDelegationToken.java      |    8 +-
 62 files changed, 3923 insertions(+), 4472 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerWithReadReplicas.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerWithReadReplicas.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerWithReadReplicas.java
index 0ea696e..3d55136 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerWithReadReplicas.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerWithReadReplicas.java
@@ -20,14 +20,17 @@ package org.apache.hadoop.hbase.client;
 
 import java.io.IOException;
 import java.io.InterruptedIOException;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
 import java.util.concurrent.CancellationException;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
+import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
@@ -97,7 +100,6 @@ public class RpcRetryingCallerWithReadReplicas {
       this.id = id;
       this.location = location;
       this.controller = rpcControllerFactory.newController();
-      controller.setPriority(tableName);
     }
 
     @Override
@@ -134,6 +136,11 @@ public class RpcRetryingCallerWithReadReplicas {
       setStub(cConnection.getClient(dest));
     }
 
+    private void initRpcController() {
+      controller.reset();
+      controller.setCallTimeout(callTimeout);
+      controller.setPriority(tableName);
+    }
     @Override
     protected Result rpcCall() throws Exception {
       if (controller.isCanceled()) return null;
@@ -141,16 +148,13 @@ public class RpcRetryingCallerWithReadReplicas {
         throw new InterruptedIOException();
       }
       byte[] reg = location.getRegionInfo().getRegionName();
-      ClientProtos.GetRequest request =
-          RequestConverter.buildGetRequest(reg, get);
-      // Presumption that we are passed a PayloadCarryingRpcController here!
-      HBaseRpcController pcrc = (HBaseRpcController)controller;
-      pcrc.setCallTimeout(callTimeout);
+      ClientProtos.GetRequest request = RequestConverter.buildGetRequest(reg, get);
+      initRpcController();
       ClientProtos.GetResponse response = getStub().get(controller, request);
       if (response == null) {
         return null;
       }
-      return ProtobufUtil.toResult(response.getResult(), pcrc.cellScanner());
+      return ProtobufUtil.toResult(response.getResult(), controller.cellScanner());
     }
 
     @Override
@@ -183,7 +187,7 @@ public class RpcRetryingCallerWithReadReplicas {
 
     RegionLocations rl = getRegionLocations(true, (isTargetReplicaSpecified ? get.getReplicaId()
         : RegionReplicaUtil.DEFAULT_REPLICA_ID), cConnection, tableName, get.getRow());
-    ResultBoundedCompletionService<Result> cs =
+   final ResultBoundedCompletionService<Result> cs =
         new ResultBoundedCompletionService<Result>(this.rpcRetryingCallerFactory, pool, rl.size());
 
     if(isTargetReplicaSpecified) {
@@ -207,7 +211,6 @@ public class RpcRetryingCallerWithReadReplicas {
       // submit call for the all of the secondaries at once
       addCallsForReplica(cs, rl, 1, rl.size() - 1);
     }
-
     try {
       try {
         long start = EnvironmentEdgeManager.currentTime();

http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
index 6cb0786..098ad3c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
@@ -18,54 +18,106 @@
 
 package org.apache.hadoop.hbase.ipc;
 
+import static org.apache.hadoop.hbase.ipc.IPCUtil.toIOE;
+import static org.apache.hadoop.hbase.ipc.IPCUtil.wrapException;
+
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
 import com.google.protobuf.BlockingRpcChannel;
 import com.google.protobuf.Descriptors;
+import com.google.protobuf.Descriptors.MethodDescriptor;
 import com.google.protobuf.Message;
+import com.google.protobuf.RpcCallback;
+import com.google.protobuf.RpcChannel;
 import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
 
+import io.netty.util.HashedWheelTimer;
+
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.SocketAddress;
 import java.net.UnknownHostException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.MetricsConnection;
 import org.apache.hadoop.hbase.codec.Codec;
 import org.apache.hadoop.hbase.codec.KeyValueCodec;
+import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos.TokenIdentifier.Kind;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.UserProvider;
+import org.apache.hadoop.hbase.security.token.AuthenticationTokenSelector;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.PoolMap;
+import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.hadoop.security.token.TokenSelector;
 
 /**
  * Provides the basics for a RpcClient implementation like configuration and Logging.
+ * <p>
+ * Locking schema of the current IPC implementation
+ * <ul>
+ * <li>There is a lock in {@link AbstractRpcClient} to protect the fetching or creating
+ * connection.</li>
+ * <li>There is a lock in {@link Call} to make sure that we can only finish the call once.</li>
+ * <li>The same for {@link HBaseRpcController} as {@link Call}. And see the comment of
+ * {@link HBaseRpcController#notifyOnCancel(RpcCallback, HBaseRpcController.CancellationCallback)}
+ * of how to deal with cancel.</li>
+ * <li>For connection implementation, the construction of a connection should be as fast as possible
+ * because the creation is protected under a lock. Connect to remote side when needed. There is no
+ * forced locking schema for a connection implementation.</li>
+ * <li>For the locking order, the {@link Call} and {@link HBaseRpcController}'s lock should be held
+ * at last. So the callbacks in {@link Call} and {@link HBaseRpcController} should be execute
+ * outside the lock in {@link Call} and {@link HBaseRpcController} which means the implementations
+ * of the callbacks are free to hold any lock.</li>
+ * </ul>
  */
 @InterfaceAudience.Private
-public abstract class AbstractRpcClient implements RpcClient {
+public abstract class AbstractRpcClient<T extends RpcConnection> implements RpcClient {
   // Log level is being changed in tests
   public static final Log LOG = LogFactory.getLog(AbstractRpcClient.class);
 
+  protected static final HashedWheelTimer WHEEL_TIMER = new HashedWheelTimer(
+      Threads.newDaemonThreadFactory("RpcClient-timer"), 10, TimeUnit.MILLISECONDS);
+
+  private static final ScheduledExecutorService IDLE_CONN_SWEEPER = Executors
+      .newScheduledThreadPool(1, Threads.newDaemonThreadFactory("Idle-Rpc-Conn-Sweeper"));
+
+  protected final static Map<Kind, TokenSelector<? extends TokenIdentifier>> TOKEN_HANDLERS = new HashMap<>();
+
+  static {
+    TOKEN_HANDLERS.put(Kind.HBASE_AUTH_TOKEN, new AuthenticationTokenSelector());
+  }
+
+  protected boolean running = true; // if client runs
+
   protected final Configuration conf;
-  protected String clusterId;
+  protected final String clusterId;
   protected final SocketAddress localAddr;
   protected final MetricsConnection metrics;
 
-  protected UserProvider userProvider;
+  protected final UserProvider userProvider;
   protected final CellBlockBuilder cellBlockBuilder;
 
   protected final int minIdleTimeBeforeClose; // if the connection is idle for more than this
   // time (in ms), it will be closed at any moment.
-  protected final int maxRetries; //the max. no. of retries for socket connections
+  protected final int maxRetries; // the max. no. of retries for socket connections
   protected final long failureSleep; // Time to sleep before retry on failure.
   protected final boolean tcpNoDelay; // if T then disable Nagle's Algorithm
   protected final boolean tcpKeepAlive; // if T then use keepalives
@@ -73,13 +125,20 @@ public abstract class AbstractRpcClient implements RpcClient {
   protected final CompressionCodec compressor;
   protected final boolean fallbackAllowed;
 
+  protected final FailedServers failedServers;
+
   protected final int connectTO;
   protected final int readTO;
   protected final int writeTO;
 
+  protected final PoolMap<ConnectionId, T> connections;
+
+  private final AtomicInteger callIdCnt = new AtomicInteger(0);
+
+  private final ScheduledFuture<?> cleanupIdleConnectionTask;
+
   /**
    * Construct an IPC client for the cluster <code>clusterId</code>
-   *
    * @param conf configuration
    * @param clusterId the cluster id
    * @param localAddr client socket bind address.
@@ -92,7 +151,7 @@ public abstract class AbstractRpcClient implements RpcClient {
     this.tcpKeepAlive = conf.getBoolean("hbase.ipc.client.tcpkeepalive", true);
     this.clusterId = clusterId != null ? clusterId : HConstants.CLUSTER_ID_DEFAULT;
     this.failureSleep = conf.getLong(HConstants.HBASE_CLIENT_PAUSE,
-        HConstants.DEFAULT_HBASE_CLIENT_PAUSE);
+      HConstants.DEFAULT_HBASE_CLIENT_PAUSE);
     this.maxRetries = conf.getInt("hbase.ipc.client.connect.max.retries", 0);
     this.tcpNoDelay = conf.getBoolean("hbase.ipc.client.tcpnodelay", true);
     this.cellBlockBuilder = new CellBlockBuilder(conf);
@@ -102,31 +161,53 @@ public abstract class AbstractRpcClient implements RpcClient {
     this.codec = getCodec();
     this.compressor = getCompressor(conf);
     this.fallbackAllowed = conf.getBoolean(IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_KEY,
-        IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_DEFAULT);
+      IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_DEFAULT);
+    this.failedServers = new FailedServers(conf);
     this.connectTO = conf.getInt(SOCKET_TIMEOUT_CONNECT, DEFAULT_SOCKET_TIMEOUT_CONNECT);
     this.readTO = conf.getInt(SOCKET_TIMEOUT_READ, DEFAULT_SOCKET_TIMEOUT_READ);
     this.writeTO = conf.getInt(SOCKET_TIMEOUT_WRITE, DEFAULT_SOCKET_TIMEOUT_WRITE);
     this.metrics = metrics;
 
-    // login the server principal (if using secure Hadoop)
+    this.connections = new PoolMap<>(getPoolType(conf), getPoolSize(conf));
+
+    this.cleanupIdleConnectionTask = IDLE_CONN_SWEEPER.scheduleAtFixedRate(new Runnable() {
+
+      @Override
+      public void run() {
+        cleanupIdleConnections();
+      }
+    }, minIdleTimeBeforeClose, minIdleTimeBeforeClose, TimeUnit.MILLISECONDS);
+
     if (LOG.isDebugEnabled()) {
-      LOG.debug("Codec=" + this.codec + ", compressor=" + this.compressor +
-          ", tcpKeepAlive=" + this.tcpKeepAlive +
-          ", tcpNoDelay=" + this.tcpNoDelay +
-          ", connectTO=" + this.connectTO +
-          ", readTO=" + this.readTO +
-          ", writeTO=" + this.writeTO +
-          ", minIdleTimeBeforeClose=" + this.minIdleTimeBeforeClose +
-          ", maxRetries=" + this.maxRetries +
-          ", fallbackAllowed=" + this.fallbackAllowed +
-          ", bind address=" + (this.localAddr != null ? this.localAddr : "null"));
+      LOG.debug("Codec=" + this.codec + ", compressor=" + this.compressor + ", tcpKeepAlive="
+          + this.tcpKeepAlive + ", tcpNoDelay=" + this.tcpNoDelay + ", connectTO=" + this.connectTO
+          + ", readTO=" + this.readTO + ", writeTO=" + this.writeTO + ", minIdleTimeBeforeClose="
+          + this.minIdleTimeBeforeClose + ", maxRetries=" + this.maxRetries + ", fallbackAllowed="
+          + this.fallbackAllowed + ", bind address="
+          + (this.localAddr != null ? this.localAddr : "null"));
+    }
+  }
+
+  private void cleanupIdleConnections() {
+    long closeBeforeTime = EnvironmentEdgeManager.currentTime() - minIdleTimeBeforeClose;
+    synchronized (connections) {
+      for (T conn : connections.values()) {
+        // remove connection if it has not been chosen by anyone for more than maxIdleTime, and the
+        // connection itself has already shutdown. The latter check is because that we may still
+        // have some pending calls on connection so we should not shutdown the connection outside.
+        // The connection itself will disconnect if there is no pending call for maxIdleTime.
+        if (conn.getLastTouched() < closeBeforeTime && !conn.isActive()) {
+          LOG.info("Cleanup idle connection to " + conn.remoteId().address);
+          connections.removeValue(conn.remoteId(), conn);
+        }
+      }
     }
   }
 
   @VisibleForTesting
   public static String getDefaultCodec(final Configuration c) {
     // If "hbase.client.default.rpc.codec" is empty string -- you can't set it to null because
-    // Configuration will complain -- then no default codec (and we'll pb everything).  Else
+    // Configuration will complain -- then no default codec (and we'll pb everything). Else
     // default is KeyValueCodec
     return c.get(DEFAULT_CODEC_CLASS, KeyValueCodec.class.getCanonicalName());
   }
@@ -143,7 +224,7 @@ public abstract class AbstractRpcClient implements RpcClient {
       return null;
     }
     try {
-      return (Codec)Class.forName(className).newInstance();
+      return (Codec) Class.forName(className).newInstance();
     } catch (Exception e) {
       throw new RuntimeException("Failed getting codec " + className, e);
     }
@@ -154,6 +235,12 @@ public abstract class AbstractRpcClient implements RpcClient {
     return this.codec != null;
   }
 
+  // for writing tests that want to throw exception when connecting.
+  @VisibleForTesting
+  boolean isTcpNoDelay() {
+    return tcpNoDelay;
+  }
+
   /**
    * Encapsulate the ugly casting and RuntimeException conversion in private method.
    * @param conf configuration
@@ -165,163 +252,297 @@ public abstract class AbstractRpcClient implements RpcClient {
       return null;
     }
     try {
-      return (CompressionCodec)Class.forName(className).newInstance();
+      return (CompressionCodec) Class.forName(className).newInstance();
     } catch (Exception e) {
       throw new RuntimeException("Failed getting compressor " + className, e);
     }
   }
 
   /**
-   * Return the pool type specified in the configuration, which must be set to
-   * either {@link org.apache.hadoop.hbase.util.PoolMap.PoolType#RoundRobin} or
-   * {@link org.apache.hadoop.hbase.util.PoolMap.PoolType#ThreadLocal},
-   * otherwise default to the former.
-   *
-   * For applications with many user threads, use a small round-robin pool. For
-   * applications with few user threads, you may want to try using a
-   * thread-local pool. In any case, the number of {@link org.apache.hadoop.hbase.ipc.RpcClient}
-   * instances should not exceed the operating system's hard limit on the number of
-   * connections.
-   *
+   * Return the pool type specified in the configuration, which must be set to either
+   * {@link org.apache.hadoop.hbase.util.PoolMap.PoolType#RoundRobin} or
+   * {@link org.apache.hadoop.hbase.util.PoolMap.PoolType#ThreadLocal}, otherwise default to the
+   * former. For applications with many user threads, use a small round-robin pool. For applications
+   * with few user threads, you may want to try using a thread-local pool. In any case, the number
+   * of {@link org.apache.hadoop.hbase.ipc.RpcClient} instances should not exceed the operating
+   * system's hard limit on the number of connections.
    * @param config configuration
    * @return either a {@link org.apache.hadoop.hbase.util.PoolMap.PoolType#RoundRobin} or
    *         {@link org.apache.hadoop.hbase.util.PoolMap.PoolType#ThreadLocal}
    */
-  protected static PoolMap.PoolType getPoolType(Configuration config) {
-    return PoolMap.PoolType
-        .valueOf(config.get(HConstants.HBASE_CLIENT_IPC_POOL_TYPE), PoolMap.PoolType.RoundRobin,
-            PoolMap.PoolType.ThreadLocal);
+  private static PoolMap.PoolType getPoolType(Configuration config) {
+    return PoolMap.PoolType.valueOf(config.get(HConstants.HBASE_CLIENT_IPC_POOL_TYPE),
+      PoolMap.PoolType.RoundRobin, PoolMap.PoolType.ThreadLocal);
   }
 
   /**
-   * Return the pool size specified in the configuration, which is applicable only if
-   * the pool type is {@link org.apache.hadoop.hbase.util.PoolMap.PoolType#RoundRobin}.
-   *
+   * Return the pool size specified in the configuration, which is applicable only if the pool type
+   * is {@link org.apache.hadoop.hbase.util.PoolMap.PoolType#RoundRobin}.
    * @param config configuration
    * @return the maximum pool size
    */
-  protected static int getPoolSize(Configuration config) {
+  private static int getPoolSize(Configuration config) {
     return config.getInt(HConstants.HBASE_CLIENT_IPC_POOL_SIZE, 1);
   }
 
+  private int nextCallId() {
+    int id, next;
+    do {
+      id = callIdCnt.get();
+      next = id < Integer.MAX_VALUE ? id + 1 : 0;
+    } while (!callIdCnt.compareAndSet(id, next));
+    return id;
+  }
+
   /**
    * Make a blocking call. Throws exceptions if there are network problems or if the remote code
    * threw an exception.
-   *
    * @param ticket Be careful which ticket you pass. A new user will mean a new Connection.
-   *               {@link UserProvider#getCurrent()} makes a new instance of User each time so
-   *               will be a
-   *               new Connection each time.
+   *          {@link UserProvider#getCurrent()} makes a new instance of User each time so will be a
+   *          new Connection each time.
    * @return A pair with the Message response and the Cell data (if any).
    */
-  private Message callBlockingMethod(Descriptors.MethodDescriptor md, HBaseRpcController pcrc,
+  private Message callBlockingMethod(Descriptors.MethodDescriptor md, HBaseRpcController hrc,
       Message param, Message returnType, final User ticket, final InetSocketAddress isa)
       throws ServiceException {
-    if (pcrc == null) {
-      pcrc = new HBaseRpcControllerImpl();
+    BlockingRpcCallback<Message> done = new BlockingRpcCallback<>();
+    callMethod(md, hrc, param, returnType, ticket, isa, done);
+    Message val;
+    try {
+      val = done.get();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    if (hrc.failed()) {
+      throw new ServiceException(hrc.getFailed());
+    } else {
+      return val;
     }
+  }
 
-    Pair<Message, CellScanner> val;
-    try {
-      final MetricsConnection.CallStats cs = MetricsConnection.newCallStats();
-      cs.setStartTime(EnvironmentEdgeManager.currentTime());
-      val = call(pcrc, md, param, returnType, ticket, isa, cs);
-      // Shove the results into controller so can be carried across the proxy/pb service void.
-      pcrc.setCellScanner(val.getSecond());
-
-      cs.setCallTimeMs(EnvironmentEdgeManager.currentTime() - cs.getStartTime());
-      if (metrics != null) {
-        metrics.updateRpc(md, param, cs);
+  /**
+   * Get a connection from the pool, or create a new one and add it to the pool. Connections to a
+   * given host/port are reused.
+   */
+  private T getConnection(ConnectionId remoteId) throws IOException {
+    if (failedServers.isFailedServer(remoteId.getAddress())) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Not trying to connect to " + remoteId.address
+            + " this server is in the failed servers list");
       }
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("Call: " + md.getName() + ", callTime: " + cs.getCallTimeMs() + "ms");
+      throw new FailedServerException(
+          "This server is in the failed servers list: " + remoteId.address);
+    }
+    T conn;
+    synchronized (connections) {
+      if (!running) {
+        throw new StoppedRpcClientException();
       }
-      return val.getFirst();
-    } catch (Throwable e) {
-      throw new ServiceException(e);
+      conn = connections.get(remoteId);
+      if (conn == null) {
+        conn = createConnection(remoteId);
+        connections.put(remoteId, conn);
+      }
+      conn.setLastTouched(EnvironmentEdgeManager.currentTime());
     }
+    return conn;
   }
 
   /**
-   * Make a call, passing <code>param</code>, to the IPC server running at
-   * <code>address</code> which is servicing the <code>protocol</code> protocol,
-   * with the <code>ticket</code> credentials, returning the value.
-   * Throws exceptions if there are network problems or if the remote code
-   * threw an exception.
-   *
-   * @param ticket Be careful which ticket you pass. A new user will mean a new Connection.
-   *               {@link UserProvider#getCurrent()} makes a new instance of User each time so
-   *               will be a
-   *               new Connection each time.
-   * @return A pair with the Message response and the Cell data (if any).
-   * @throws InterruptedException if call is interrupted
-   * @throws java.io.IOException if transport failed
+   * Not connected.
    */
-  protected abstract Pair<Message, CellScanner> call(HBaseRpcController pcrc,
-      Descriptors.MethodDescriptor md, Message param, Message returnType, User ticket,
-      InetSocketAddress isa, MetricsConnection.CallStats callStats)
-      throws IOException, InterruptedException;
+  protected abstract T createConnection(ConnectionId remoteId) throws IOException;
 
-  @Override
-  public BlockingRpcChannel createBlockingRpcChannel(final ServerName sn, final User ticket,
-      int defaultOperationTimeout) throws UnknownHostException {
-    return new BlockingRpcChannelImplementation(this, sn, ticket, defaultOperationTimeout);
+  private void onCallFinished(Call call, HBaseRpcController hrc, InetSocketAddress addr,
+      RpcCallback<Message> callback) {
+    call.callStats.setCallTimeMs(EnvironmentEdgeManager.currentTime() - call.getStartTime());
+    if (metrics != null) {
+      metrics.updateRpc(call.md, call.param, call.callStats);
+    }
+    if (LOG.isTraceEnabled()) {
+      LOG.trace(
+        "Call: " + call.md.getName() + ", callTime: " + call.callStats.getCallTimeMs() + "ms");
+    }
+    if (call.error != null) {
+      if (call.error instanceof RemoteException) {
+        call.error.fillInStackTrace();
+        hrc.setFailed(call.error);
+      } else {
+        hrc.setFailed(wrapException(addr, call.error));
+      }
+      callback.run(null);
+    } else {
+      hrc.setDone(call.cells);
+      callback.run(call.response);
+    }
+  }
+
+  private void callMethod(final Descriptors.MethodDescriptor md, final HBaseRpcController hrc,
+      final Message param, Message returnType, final User ticket, final InetSocketAddress addr,
+      final RpcCallback<Message> callback) {
+    final MetricsConnection.CallStats cs = MetricsConnection.newCallStats();
+    cs.setStartTime(EnvironmentEdgeManager.currentTime());
+    Call call = new Call(nextCallId(), md, param, hrc.cellScanner(), returnType,
+        hrc.getCallTimeout(), hrc.getPriority(), new RpcCallback<Call>() {
+
+          @Override
+          public void run(Call call) {
+            onCallFinished(call, hrc, addr, callback);
+          }
+        }, cs);
+    ConnectionId remoteId = new ConnectionId(ticket, md.getService().getName(), addr);
+    try {
+      T connection = getConnection(remoteId);
+      connection.sendRequest(call, hrc);
+    } catch (Exception e) {
+      call.setException(toIOE(e));
+    }
+  }
+
+  private InetSocketAddress createAddr(ServerName sn) throws UnknownHostException {
+    InetSocketAddress addr = new InetSocketAddress(sn.getHostname(), sn.getPort());
+    if (addr.isUnresolved()) {
+      throw new UnknownHostException("can not resolve " + sn.getServerName());
+    }
+    return addr;
   }
 
   /**
-   * Configure a payload carrying controller
-   * @param controller to configure
-   * @param channelOperationTimeout timeout for operation
-   * @return configured payload controller
+   * Interrupt the connections to the given ip:port server. This should be called if the server is
+   * known as actually dead. This will not prevent current operation to be retried, and, depending
+   * on their own behavior, they may retry on the same server. This can be a feature, for example at
+   * startup. In any case, they're likely to get connection refused (if the process died) or no
+   * route to host: i.e. their next retries should be faster and with a safe exception.
    */
-  static HBaseRpcController configurePayloadCarryingRpcController(
-      RpcController controller, int channelOperationTimeout) {
-    HBaseRpcController pcrc;
-    if (controller != null && controller instanceof HBaseRpcController) {
-      pcrc = (HBaseRpcController) controller;
-      if (!pcrc.hasCallTimeout()) {
-        pcrc.setCallTimeout(channelOperationTimeout);
+  @Override
+  public void cancelConnections(ServerName sn) {
+    synchronized (connections) {
+      for (T connection : connections.values()) {
+        ConnectionId remoteId = connection.remoteId();
+        if (remoteId.address.getPort() == sn.getPort()
+            && remoteId.address.getHostName().equals(sn.getHostname())) {
+          LOG.info("The server on " + sn.toString() + " is dead - stopping the connection "
+              + connection.remoteId);
+          connection.shutdown();
+        }
       }
-    } else {
-      pcrc = new HBaseRpcControllerImpl();
-      pcrc.setCallTimeout(channelOperationTimeout);
     }
-    return pcrc;
+  }
+
+  protected abstract void closeInternal();
+
+  @Override
+  public void close() {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Stopping rpc client");
+    }
+    Collection<T> connToClose;
+    synchronized (connections) {
+      if (!running) {
+        return;
+      }
+      running = false;
+      connToClose = connections.values();
+      connections.clear();
+    }
+    cleanupIdleConnectionTask.cancel(true);
+    for (T conn : connToClose) {
+      conn.shutdown();
+    }
+    closeInternal();
+  }
+
+  @Override
+  public BlockingRpcChannel createBlockingRpcChannel(final ServerName sn, final User ticket,
+      int rpcTimeout) throws UnknownHostException {
+    return new BlockingRpcChannelImplementation(this, createAddr(sn), ticket, rpcTimeout);
+  }
+
+  @Override
+  public RpcChannel createRpcChannel(ServerName sn, User user, int rpcTimeout)
+      throws UnknownHostException {
+    return new RpcChannelImplementation(this, createAddr(sn), user, rpcTimeout);
+  }
+
+  private static class AbstractRpcChannel {
+
+    protected final InetSocketAddress addr;
+
+    protected final AbstractRpcClient<?> rpcClient;
+
+    protected final User ticket;
+
+    protected final int rpcTimeout;
+
+    protected AbstractRpcChannel(AbstractRpcClient<?> rpcClient, InetSocketAddress addr,
+        User ticket, int rpcTimeout) {
+      this.addr = addr;
+      this.rpcClient = rpcClient;
+      this.ticket = ticket;
+      this.rpcTimeout = rpcTimeout;
+    }
+
+    /**
+     * Configure an rpc controller
+     * @param controller to configure
+     * @return configured rpc controller
+     */
+    protected HBaseRpcController configureRpcController(RpcController controller) {
+      HBaseRpcController hrc;
+      // TODO: Ideally we should not use an RpcController other than HBaseRpcController at client
+      // side. And now we may use ServerRpcController.
+      if (controller != null && controller instanceof HBaseRpcController) {
+        hrc = (HBaseRpcController) controller;
+        if (!hrc.hasCallTimeout()) {
+          hrc.setCallTimeout(rpcTimeout);
+        }
+      } else {
+        hrc = new HBaseRpcControllerImpl();
+        hrc.setCallTimeout(rpcTimeout);
+      }
+      return hrc;
+    }
   }
 
   /**
    * Blocking rpc channel that goes via hbase rpc.
    */
   @VisibleForTesting
-  public static class BlockingRpcChannelImplementation implements BlockingRpcChannel {
-    private final InetSocketAddress isa;
-    private final AbstractRpcClient rpcClient;
-    private final User ticket;
-    private final int channelOperationTimeout;
+  public static class BlockingRpcChannelImplementation extends AbstractRpcChannel
+      implements BlockingRpcChannel {
 
-    /**
-     * @param channelOperationTimeout - the default timeout when no timeout is given
-     */
-    protected BlockingRpcChannelImplementation(final AbstractRpcClient rpcClient,
-        final ServerName sn, final User ticket, int channelOperationTimeout)
-        throws UnknownHostException {
-      this.isa = new InetSocketAddress(sn.getHostname(), sn.getPort());
-      if (this.isa.isUnresolved()) {
-        throw new UnknownHostException(sn.getHostname());
-      }
-      this.rpcClient = rpcClient;
-      this.ticket = ticket;
-      this.channelOperationTimeout = channelOperationTimeout;
+    protected BlockingRpcChannelImplementation(AbstractRpcClient<?> rpcClient,
+        InetSocketAddress addr, User ticket, int rpcTimeout) {
+      super(rpcClient, addr, ticket, rpcTimeout);
     }
 
     @Override
     public Message callBlockingMethod(Descriptors.MethodDescriptor md, RpcController controller,
         Message param, Message returnType) throws ServiceException {
-      HBaseRpcController pcrc = configurePayloadCarryingRpcController(
-          controller,
-          channelOperationTimeout);
+      return rpcClient.callBlockingMethod(md, configureRpcController(controller),
+        param, returnType, ticket, addr);
+    }
+  }
 
-      return this.rpcClient.callBlockingMethod(md, pcrc, param, returnType, this.ticket, this.isa);
+  /**
+   * Async rpc channel that goes via hbase rpc.
+   */
+  public static class RpcChannelImplementation extends AbstractRpcChannel implements RpcChannel {
+
+    protected RpcChannelImplementation(AbstractRpcClient<?> rpcClient, InetSocketAddress addr,
+        User ticket, int rpcTimeout) throws UnknownHostException {
+      super(rpcClient, addr, ticket, rpcTimeout);
+    }
+
+    @Override
+    public void callMethod(MethodDescriptor md, RpcController controller, Message param,
+        Message returnType, RpcCallback<Message> done) {
+      // This method does not throw any exceptions, so the caller must provide a
+      // HBaseRpcController which is used to pass the exceptions.
+      this.rpcClient.callMethod(md,
+        configureRpcController(Preconditions.checkNotNull(controller,
+          "RpcController can not be null for async rpc call")),
+        param, returnType, ticket, addr, done);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncCall.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncCall.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncCall.java
deleted file mode 100644
index 33536df..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncCall.java
+++ /dev/null
@@ -1,204 +0,0 @@
-/**
- * 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.ipc;
-
-import com.google.protobuf.Descriptors;
-import com.google.protobuf.Message;
-
-import io.netty.util.concurrent.DefaultPromise;
-
-import java.io.IOException;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.CellScanner;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.client.MetricsConnection;
-import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.ExceptionUtil;
-import org.apache.hadoop.ipc.RemoteException;
-
-/**
- * Represents an Async Hbase call and its response.
- *
- * Responses are passed on to its given doneHandler and failures to the rpcController
- *
- * @param <T> Type of message returned
- * @param <M> Message returned in communication to be converted
- */
-@InterfaceAudience.Private
-public class AsyncCall<M extends Message, T> extends DefaultPromise<T> {
-  private static final Log LOG = LogFactory.getLog(AsyncCall.class.getName());
-
-  final int id;
-
-  private final AsyncRpcChannel channel;
-
-  final Descriptors.MethodDescriptor method;
-  final Message param;
-  final Message responseDefaultType;
-
-  private final MessageConverter<M,T> messageConverter;
-  private final IOExceptionConverter exceptionConverter;
-
-  final long rpcTimeout;
-
-  // For only the request
-  private final CellScanner cellScanner;
-  private final int priority;
-
-  final MetricsConnection clientMetrics;
-  final MetricsConnection.CallStats callStats;
-
-  /**
-   * Constructor
-   *
-   * @param channel             which initiated call
-   * @param connectId           connection id
-   * @param md                  the method descriptor
-   * @param param               parameters to send to Server
-   * @param cellScanner         cellScanner containing cells to send as request
-   * @param responseDefaultType the default response type
-   * @param messageConverter    converts the messages to what is the expected output
-   * @param exceptionConverter  converts exceptions to expected format. Can be null
-   * @param rpcTimeout          timeout for this call in ms
-   * @param priority            for this request
-   * @param metrics             MetricsConnection to which the metrics are stored for this request
-   */
-  public AsyncCall(AsyncRpcChannel channel, int connectId, Descriptors.MethodDescriptor
-        md, Message param, CellScanner cellScanner, M responseDefaultType, MessageConverter<M, T>
-        messageConverter, IOExceptionConverter exceptionConverter, long rpcTimeout, int priority,
-      MetricsConnection metrics) {
-    super(channel.getEventExecutor());
-    this.channel = channel;
-
-    this.id = connectId;
-
-    this.method = md;
-    this.param = param;
-    this.responseDefaultType = responseDefaultType;
-
-    this.messageConverter = messageConverter;
-    this.exceptionConverter = exceptionConverter;
-
-    this.rpcTimeout = rpcTimeout;
-
-    this.priority = priority;
-    this.cellScanner = cellScanner;
-
-    this.callStats = MetricsConnection.newCallStats();
-    callStats.setStartTime(EnvironmentEdgeManager.currentTime());
-
-    this.clientMetrics = metrics;
-  }
-
-  /**
-   * Get the start time
-   *
-   * @return start time for the call
-   */
-  public long getStartTime() {
-    return this.callStats.getStartTime();
-  }
-
-  @Override
-  public String toString() {
-    return "callId=" + this.id + ", method=" + this.method.getName() +
-      ", rpcTimeout=" + this.rpcTimeout + ", param {" +
-      (this.param != null ? ProtobufUtil.getShortTextFormat(this.param) : "") + "}";
-  }
-
-  /**
-   * Set success with a cellBlockScanner
-   *
-   * @param value            to set
-   * @param cellBlockScanner to set
-   */
-  public void setSuccess(M value, CellScanner cellBlockScanner) {
-    callStats.setCallTimeMs(EnvironmentEdgeManager.currentTime() - callStats.getStartTime());
-
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("Call: " + method.getName() + ", callTime: " + callStats.getCallTimeMs() + "ms");
-    }
-
-    if (clientMetrics != null) {
-      clientMetrics.updateRpc(method, param, callStats);
-    }
-
-    try {
-      this.setSuccess(
-          this.messageConverter.convert(value, cellBlockScanner)
-      );
-    } catch (IOException e) {
-      this.setFailed(e);
-    }
-  }
-
-  /**
-   * Set failed
-   *
-   * @param exception to set
-   */
-  public void setFailed(IOException exception) {
-    if (ExceptionUtil.isInterrupt(exception)) {
-      exception = ExceptionUtil.asInterrupt(exception);
-    }
-    if (exception instanceof RemoteException) {
-      exception = ((RemoteException) exception).unwrapRemoteException();
-    }
-
-    if (this.exceptionConverter != null) {
-      exception = this.exceptionConverter.convert(exception);
-    }
-
-    this.setFailure(exception);
-  }
-
-  /**
-   * Get the rpc timeout
-   *
-   * @return current timeout for this call
-   */
-  public long getRpcTimeout() {
-    return rpcTimeout;
-  }
-
-
-  /**
-   * @return Priority for this call
-   */
-  public int getPriority() {
-    return priority;
-  }
-
-  /**
-   * Get the cellScanner for this request.
-   * @return CellScanner
-   */
-  public CellScanner cellScanner() {
-    return cellScanner;
-  }
-
-  @Override
-  public boolean cancel(boolean mayInterupt){
-    this.channel.removePendingCall(this.id);
-    return super.cancel(mayInterupt);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcChannel.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcChannel.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcChannel.java
deleted file mode 100644
index 2ec5adc..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcChannel.java
+++ /dev/null
@@ -1,768 +0,0 @@
-/**
- * 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.ipc;
-
-import com.google.protobuf.Descriptors;
-import com.google.protobuf.Message;
-
-import io.netty.bootstrap.Bootstrap;
-import io.netty.buffer.ByteBuf;
-import io.netty.buffer.ByteBufOutputStream;
-import io.netty.channel.Channel;
-import io.netty.channel.ChannelFuture;
-import io.netty.channel.ChannelFutureListener;
-import io.netty.channel.EventLoop;
-import io.netty.handler.codec.LengthFieldBasedFrameDecoder;
-import io.netty.util.Timeout;
-import io.netty.util.TimerTask;
-import io.netty.util.concurrent.GenericFutureListener;
-
-import java.io.IOException;
-import java.net.ConnectException;
-import java.net.InetSocketAddress;
-import java.nio.ByteBuffer;
-import java.security.PrivilegedExceptionAction;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Locale;
-import java.util.Map;
-import java.util.Random;
-import java.util.concurrent.TimeUnit;
-
-import javax.security.sasl.SaslException;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.CellScanner;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.exceptions.ConnectionClosingException;
-import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos;
-import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos.TokenIdentifier.Kind;
-import org.apache.hadoop.hbase.protobuf.generated.RPCProtos;
-import org.apache.hadoop.hbase.protobuf.generated.TracingProtos;
-import org.apache.hadoop.hbase.security.AuthMethod;
-import org.apache.hadoop.hbase.security.SaslClientHandler;
-import org.apache.hadoop.hbase.security.SaslUtil;
-import org.apache.hadoop.hbase.security.SecurityInfo;
-import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.security.token.AuthenticationTokenSelector;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hadoop.security.SecurityUtil;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.security.token.TokenIdentifier;
-import org.apache.hadoop.security.token.TokenSelector;
-import org.apache.htrace.Span;
-import org.apache.htrace.Trace;
-
-/**
- * Netty RPC channel
- */
-@InterfaceAudience.Private
-public class AsyncRpcChannel {
-  private static final Log LOG = LogFactory.getLog(AsyncRpcChannel.class.getName());
-
-  private static final int MAX_SASL_RETRIES = 5;
-
-  protected final static Map<Kind, TokenSelector<? extends TokenIdentifier>> TOKEN_HANDDLERS
-    = new HashMap<>();
-
-  static {
-    TOKEN_HANDDLERS.put(AuthenticationProtos.TokenIdentifier.Kind.HBASE_AUTH_TOKEN,
-      new AuthenticationTokenSelector());
-  }
-
-  final AsyncRpcClient client;
-
-  // Contains the channel to work with.
-  // Only exists when connected
-  private Channel channel;
-
-  String name;
-  final User ticket;
-  final String serviceName;
-  final InetSocketAddress address;
-
-  private int failureCounter = 0;
-
-  boolean useSasl;
-  AuthMethod authMethod;
-  private int reloginMaxBackoff;
-  private Token<? extends TokenIdentifier> token;
-  private String serverPrincipal;
-
-  // NOTE: closed and connected flags below are only changed when a lock on pendingCalls
-  private final Map<Integer, AsyncCall> pendingCalls = new HashMap<Integer, AsyncCall>();
-  private boolean connected = false;
-  private boolean closed = false;
-
-  private Timeout cleanupTimer;
-
-  private final TimerTask timeoutTask = new TimerTask() {
-    @Override
-    public void run(Timeout timeout) throws Exception {
-      cleanupCalls();
-    }
-  };
-
-  /**
-   * Constructor for netty RPC channel
-   * @param bootstrap to construct channel on
-   * @param client to connect with
-   * @param ticket of user which uses connection
-   * @param serviceName name of service to connect to
-   * @param address to connect to
-   */
-  public AsyncRpcChannel(Bootstrap bootstrap, final AsyncRpcClient client, User ticket,
-      String serviceName, InetSocketAddress address) {
-    this.client = client;
-
-    this.ticket = ticket;
-    this.serviceName = serviceName;
-    this.address = address;
-
-    this.channel = connect(bootstrap).channel();
-
-    name = ("IPC Client (" + channel.hashCode() + ") to " + address.toString()
-        + ((ticket == null) ? " from unknown user" : (" from " + ticket.getName())));
-  }
-
-  /**
-   * Connect to channel
-   * @param bootstrap to connect to
-   * @return future of connection
-   */
-  private ChannelFuture connect(final Bootstrap bootstrap) {
-    return bootstrap.remoteAddress(address).connect()
-        .addListener(new GenericFutureListener<ChannelFuture>() {
-          @Override
-          public void operationComplete(final ChannelFuture f) throws Exception {
-            if (!f.isSuccess()) {
-              retryOrClose(bootstrap, failureCounter++, client.failureSleep, f.cause());
-              return;
-            }
-            channel = f.channel();
-
-            setupAuthorization();
-
-            ByteBuf b = channel.alloc().directBuffer(6);
-            createPreamble(b, authMethod);
-            channel.writeAndFlush(b).addListener(ChannelFutureListener.CLOSE_ON_FAILURE);
-            if (useSasl) {
-              UserGroupInformation ticket = AsyncRpcChannel.this.ticket.getUGI();
-              if (authMethod == AuthMethod.KERBEROS) {
-                if (ticket != null && ticket.getRealUser() != null) {
-                  ticket = ticket.getRealUser();
-                }
-              }
-              SaslClientHandler saslHandler;
-              if (ticket == null) {
-                throw new FatalConnectionException("ticket/user is null");
-              }
-              final UserGroupInformation realTicket = ticket;
-              saslHandler = ticket.doAs(new PrivilegedExceptionAction<SaslClientHandler>() {
-                @Override
-                public SaslClientHandler run() throws IOException {
-                  return getSaslHandler(realTicket, bootstrap);
-                }
-              });
-              if (saslHandler != null) {
-                // Sasl connect is successful. Let's set up Sasl channel handler
-                channel.pipeline().addFirst(saslHandler);
-              } else {
-                // fall back to simple auth because server told us so.
-                authMethod = AuthMethod.SIMPLE;
-                useSasl = false;
-              }
-            } else {
-              startHBaseConnection(f.channel());
-            }
-          }
-        });
-  }
-
-  /**
-   * Start HBase connection
-   * @param ch channel to start connection on
-   */
-  private void startHBaseConnection(Channel ch) {
-    ch.pipeline().addLast("frameDecoder",
-      new LengthFieldBasedFrameDecoder(Integer.MAX_VALUE, 0, 4, 0, 4));
-    ch.pipeline().addLast(new AsyncServerResponseHandler(this));
-    try {
-      writeChannelHeader(ch).addListener(new GenericFutureListener<ChannelFuture>() {
-        @Override
-        public void operationComplete(ChannelFuture future) throws Exception {
-          if (!future.isSuccess()) {
-            close(future.cause());
-            return;
-          }
-          List<AsyncCall> callsToWrite;
-          synchronized (pendingCalls) {
-            connected = true;
-            callsToWrite = new ArrayList<AsyncCall>(pendingCalls.values());
-          }
-          for (AsyncCall call : callsToWrite) {
-            writeRequest(call);
-          }
-        }
-      });
-    } catch (IOException e) {
-      close(e);
-    }
-  }
-
-  private void startConnectionWithEncryption(Channel ch) {
-    // for rpc encryption, the order of ChannelInboundHandler should be:
-    // LengthFieldBasedFrameDecoder->SaslClientHandler->LengthFieldBasedFrameDecoder
-    // Don't skip the first 4 bytes for length in beforeUnwrapDecoder,
-    // SaslClientHandler will handler this
-    ch.pipeline().addFirst("beforeUnwrapDecoder",
-        new LengthFieldBasedFrameDecoder(Integer.MAX_VALUE, 0, 4, 0, 0));
-    ch.pipeline().addLast("afterUnwrapDecoder",
-        new LengthFieldBasedFrameDecoder(Integer.MAX_VALUE, 0, 4, 0, 4));
-    ch.pipeline().addLast(new AsyncServerResponseHandler(this));
-    List<AsyncCall> callsToWrite;
-    synchronized (pendingCalls) {
-      connected = true;
-      callsToWrite = new ArrayList<AsyncCall>(pendingCalls.values());
-    }
-    for (AsyncCall call : callsToWrite) {
-      writeRequest(call);
-    }
-  }
-
-  /**
-   * Get SASL handler
-   * @param bootstrap to reconnect to
-   * @return new SASL handler
-   * @throws java.io.IOException if handler failed to create
-   */
-  private SaslClientHandler getSaslHandler(final UserGroupInformation realTicket,
-      final Bootstrap bootstrap) throws IOException {
-    return new SaslClientHandler(realTicket, authMethod, token, serverPrincipal,
-        client.fallbackAllowed,
-        client.conf.get("hbase.rpc.protection",
-          SaslUtil.QualityOfProtection.AUTHENTICATION.name().toLowerCase(Locale.ROOT)),
-        getChannelHeaderBytes(authMethod),
-        new SaslClientHandler.SaslExceptionHandler() {
-          @Override
-          public void handle(int retryCount, Random random, Throwable cause) {
-            try {
-              // Handle Sasl failure. Try to potentially get new credentials
-              handleSaslConnectionFailure(retryCount, cause, realTicket);
-
-              retryOrClose(bootstrap, failureCounter++, random.nextInt(reloginMaxBackoff) + 1,
-                cause);
-            } catch (IOException | InterruptedException e) {
-              close(e);
-            }
-          }
-        }, new SaslClientHandler.SaslSuccessfulConnectHandler() {
-          @Override
-          public void onSuccess(Channel channel) {
-            startHBaseConnection(channel);
-          }
-
-          @Override
-          public void onSaslProtectionSucess(Channel channel) {
-            startConnectionWithEncryption(channel);
-          }
-        });
-  }
-
-  /**
-   * Retry to connect or close
-   * @param bootstrap to connect with
-   * @param failureCount failure count
-   * @param e exception of fail
-   */
-  private void retryOrClose(final Bootstrap bootstrap, int failureCount, long timeout,
-      Throwable e) {
-    if (failureCount < client.maxRetries) {
-      client.newTimeout(new TimerTask() {
-        @Override
-        public void run(Timeout timeout) throws Exception {
-          connect(bootstrap);
-        }
-      }, timeout, TimeUnit.MILLISECONDS);
-    } else {
-      client.failedServers.addToFailedServers(address);
-      close(e);
-    }
-  }
-
-  /**
-   * Calls method on channel
-   * @param method to call
-   * @param request to send
-   * @param cellScanner with cells to send
-   * @param responsePrototype to construct response with
-   * @param rpcTimeout timeout for request
-   * @param priority for request
-   * @return Promise for the response Message
-   */
-  public <R extends Message, O> io.netty.util.concurrent.Promise<O> callMethod(
-      final Descriptors.MethodDescriptor method,
-      final Message request,final CellScanner cellScanner,
-      R responsePrototype, MessageConverter<R, O> messageConverter, IOExceptionConverter
-      exceptionConverter, long rpcTimeout, int priority) {
-    final AsyncCall<R, O> call = new AsyncCall<>(this, client.callIdCnt.getAndIncrement(),
-        method, request, cellScanner, responsePrototype, messageConverter, exceptionConverter,
-        rpcTimeout, priority, client.metrics);
-
-    synchronized (pendingCalls) {
-      if (closed) {
-        call.setFailure(new ConnectException());
-        return call;
-      }
-      pendingCalls.put(call.id, call);
-      // Add timeout for cleanup if none is present
-      if (cleanupTimer == null && call.getRpcTimeout() > 0) {
-        cleanupTimer = client.newTimeout(timeoutTask, call.getRpcTimeout(), TimeUnit.MILLISECONDS);
-      }
-      if (!connected) {
-        return call;
-      }
-    }
-    writeRequest(call);
-    return call;
-  }
-
-  public EventLoop getEventExecutor() {
-    return this.channel.eventLoop();
-  }
-
-  AsyncCall removePendingCall(int id) {
-    synchronized (pendingCalls) {
-      return pendingCalls.remove(id);
-    }
-  }
-
-  /**
-   * Write the channel header
-   * @param channel to write to
-   * @return future of write
-   * @throws java.io.IOException on failure to write
-   */
-  private ChannelFuture writeChannelHeader(Channel channel) throws IOException {
-    RPCProtos.ConnectionHeader header = getChannelHeader(authMethod);
-    int totalSize = IPCUtil.getTotalSizeWhenWrittenDelimited(header);
-    ByteBuf b = channel.alloc().directBuffer(totalSize);
-
-    b.writeInt(header.getSerializedSize());
-    b.writeBytes(header.toByteArray());
-
-    return channel.writeAndFlush(b);
-  }
-
-  private byte[] getChannelHeaderBytes(AuthMethod authMethod) {
-    RPCProtos.ConnectionHeader header = getChannelHeader(authMethod);
-    ByteBuffer b = ByteBuffer.allocate(header.getSerializedSize() + 4);
-    b.putInt(header.getSerializedSize());
-    b.put(header.toByteArray());
-    return b.array();
-  }
-
-  private RPCProtos.ConnectionHeader getChannelHeader(AuthMethod authMethod) {
-    RPCProtos.ConnectionHeader.Builder headerBuilder = RPCProtos.ConnectionHeader.newBuilder()
-        .setServiceName(serviceName);
-
-    RPCProtos.UserInformation userInfoPB = buildUserInfo(ticket.getUGI(), authMethod);
-    if (userInfoPB != null) {
-      headerBuilder.setUserInfo(userInfoPB);
-    }
-
-    if (client.codec != null) {
-      headerBuilder.setCellBlockCodecClass(client.codec.getClass().getCanonicalName());
-    }
-    if (client.compressor != null) {
-      headerBuilder.setCellBlockCompressorClass(client.compressor.getClass().getCanonicalName());
-    }
-
-    headerBuilder.setVersionInfo(ProtobufUtil.getVersionInfo());
-    return headerBuilder.build();
-  }
-
-  /**
-   * Write request to channel
-   * @param call to write
-   */
-  private void writeRequest(final AsyncCall call) {
-    try {
-      final RPCProtos.RequestHeader.Builder requestHeaderBuilder = RPCProtos.RequestHeader
-          .newBuilder();
-      requestHeaderBuilder.setCallId(call.id).setMethodName(call.method.getName())
-          .setRequestParam(call.param != null);
-
-      if (Trace.isTracing()) {
-        Span s = Trace.currentSpan();
-        requestHeaderBuilder.setTraceInfo(TracingProtos.RPCTInfo.newBuilder()
-            .setParentId(s.getSpanId()).setTraceId(s.getTraceId()));
-      }
-
-      ByteBuffer cellBlock = client.buildCellBlock(call.cellScanner());
-      if (cellBlock != null) {
-        final RPCProtos.CellBlockMeta.Builder cellBlockBuilder = RPCProtos.CellBlockMeta
-            .newBuilder();
-        cellBlockBuilder.setLength(cellBlock.limit());
-        requestHeaderBuilder.setCellBlockMeta(cellBlockBuilder.build());
-      }
-      // Only pass priority if there one. Let zero be same as no priority.
-      if (call.getPriority() != HBaseRpcController.PRIORITY_UNSET) {
-        requestHeaderBuilder.setPriority(call.getPriority());
-      }
-      requestHeaderBuilder.setTimeout(call.rpcTimeout > Integer.MAX_VALUE ?
-          Integer.MAX_VALUE : (int)call.rpcTimeout);
-
-      RPCProtos.RequestHeader rh = requestHeaderBuilder.build();
-
-      int totalSize = IPCUtil.getTotalSizeWhenWrittenDelimited(rh, call.param);
-      if (cellBlock != null) {
-        totalSize += cellBlock.remaining();
-      }
-
-      ByteBuf b = channel.alloc().directBuffer(4 + totalSize);
-      try (ByteBufOutputStream out = new ByteBufOutputStream(b)) {
-        call.callStats.setRequestSizeBytes(IPCUtil.write(out, rh, call.param, cellBlock));
-      }
-
-      channel.writeAndFlush(b).addListener(new CallWriteListener(this, call.id));
-    } catch (IOException e) {
-      close(e);
-    }
-  }
-
-  /**
-   * Set up server authorization
-   * @throws java.io.IOException if auth setup failed
-   */
-  private void setupAuthorization() throws IOException {
-    SecurityInfo securityInfo = SecurityInfo.getInfo(serviceName);
-    this.useSasl = client.userProvider.isHBaseSecurityEnabled();
-
-    this.token = null;
-    if (useSasl && securityInfo != null) {
-      AuthenticationProtos.TokenIdentifier.Kind tokenKind = securityInfo.getTokenKind();
-      if (tokenKind != null) {
-        TokenSelector<? extends TokenIdentifier> tokenSelector = TOKEN_HANDDLERS.get(tokenKind);
-        if (tokenSelector != null) {
-          token = tokenSelector.selectToken(new Text(client.clusterId),
-            ticket.getUGI().getTokens());
-        } else if (LOG.isDebugEnabled()) {
-          LOG.debug("No token selector found for type " + tokenKind);
-        }
-      }
-      String serverKey = securityInfo.getServerPrincipal();
-      if (serverKey == null) {
-        throw new IOException("Can't obtain server Kerberos config key from SecurityInfo");
-      }
-      this.serverPrincipal = SecurityUtil.getServerPrincipal(client.conf.get(serverKey),
-        address.getAddress().getCanonicalHostName().toLowerCase(Locale.ROOT));
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("RPC Server Kerberos principal name for service=" + serviceName + " is "
-            + serverPrincipal);
-      }
-    }
-
-    if (!useSasl) {
-      authMethod = AuthMethod.SIMPLE;
-    } else if (token != null) {
-      authMethod = AuthMethod.DIGEST;
-    } else {
-      authMethod = AuthMethod.KERBEROS;
-    }
-
-    if (LOG.isDebugEnabled()) {
-      LOG.debug(
-        "Use " + authMethod + " authentication for service " + serviceName + ", sasl=" + useSasl);
-    }
-    reloginMaxBackoff = client.conf.getInt("hbase.security.relogin.maxbackoff", 5000);
-  }
-
-  /**
-   * Build the user information
-   * @param ugi User Group Information
-   * @param authMethod Authorization method
-   * @return UserInformation protobuf
-   */
-  private RPCProtos.UserInformation buildUserInfo(UserGroupInformation ugi, AuthMethod authMethod) {
-    if (ugi == null || authMethod == AuthMethod.DIGEST) {
-      // Don't send user for token auth
-      return null;
-    }
-    RPCProtos.UserInformation.Builder userInfoPB = RPCProtos.UserInformation.newBuilder();
-    if (authMethod == AuthMethod.KERBEROS) {
-      // Send effective user for Kerberos auth
-      userInfoPB.setEffectiveUser(ugi.getUserName());
-    } else if (authMethod == AuthMethod.SIMPLE) {
-      // Send both effective user and real user for simple auth
-      userInfoPB.setEffectiveUser(ugi.getUserName());
-      if (ugi.getRealUser() != null) {
-        userInfoPB.setRealUser(ugi.getRealUser().getUserName());
-      }
-    }
-    return userInfoPB.build();
-  }
-
-  /**
-   * Create connection preamble
-   * @param byteBuf to write to
-   * @param authMethod to write
-   */
-  private void createPreamble(ByteBuf byteBuf, AuthMethod authMethod) {
-    byteBuf.writeBytes(HConstants.RPC_HEADER);
-    byteBuf.writeByte(HConstants.RPC_CURRENT_VERSION);
-    byteBuf.writeByte(authMethod.code);
-  }
-
-  private void close0(Throwable e) {
-    List<AsyncCall> toCleanup;
-    synchronized (pendingCalls) {
-      if (closed) {
-        return;
-      }
-      closed = true;
-      toCleanup = new ArrayList<AsyncCall>(pendingCalls.values());
-      pendingCalls.clear();
-    }
-    IOException closeException = null;
-    if (e != null) {
-      if (e instanceof IOException) {
-        closeException = (IOException) e;
-      } else {
-        closeException = new IOException(e);
-      }
-    }
-    // log the info
-    if (LOG.isDebugEnabled() && closeException != null) {
-      LOG.debug(name + ": closing ipc connection to " + address, closeException);
-    }
-    if (cleanupTimer != null) {
-      cleanupTimer.cancel();
-      cleanupTimer = null;
-    }
-    for (AsyncCall call : toCleanup) {
-      call.setFailed(closeException != null ? closeException
-          : new ConnectionClosingException(
-              "Call id=" + call.id + " on server " + address + " aborted: connection is closing"));
-    }
-    channel.disconnect().addListener(ChannelFutureListener.CLOSE);
-    if (LOG.isDebugEnabled()) {
-      LOG.debug(name + ": closed");
-    }
-  }
-
-  /**
-   * Close connection
-   * @param e exception on close
-   */
-  public void close(final Throwable e) {
-    client.removeConnection(this);
-
-    // Move closing from the requesting thread to the channel thread
-    if (channel.eventLoop().inEventLoop()) {
-      close0(e);
-    } else {
-      channel.eventLoop().execute(new Runnable() {
-        @Override
-        public void run() {
-          close0(e);
-        }
-      });
-    }
-  }
-
-  /**
-   * Clean up calls.
-   */
-  private void cleanupCalls() {
-    List<AsyncCall> toCleanup = new ArrayList<AsyncCall>();
-    long currentTime = EnvironmentEdgeManager.currentTime();
-    long nextCleanupTaskDelay = -1L;
-    synchronized (pendingCalls) {
-      for (Iterator<AsyncCall> iter = pendingCalls.values().iterator(); iter.hasNext();) {
-        AsyncCall call = iter.next();
-        long timeout = call.getRpcTimeout();
-        if (timeout > 0) {
-          if (currentTime - call.getStartTime() >= timeout) {
-            iter.remove();
-            toCleanup.add(call);
-          } else {
-            if (nextCleanupTaskDelay < 0 || timeout < nextCleanupTaskDelay) {
-              nextCleanupTaskDelay = timeout;
-            }
-          }
-        }
-      }
-      if (nextCleanupTaskDelay > 0) {
-        cleanupTimer = client.newTimeout(timeoutTask, nextCleanupTaskDelay, TimeUnit.MILLISECONDS);
-      } else {
-        cleanupTimer = null;
-      }
-    }
-    for (AsyncCall call : toCleanup) {
-      call.setFailed(new CallTimeoutException("Call id=" + call.id + ", waitTime="
-          + (currentTime - call.getStartTime()) + ", rpcTimeout=" + call.getRpcTimeout()));
-    }
-  }
-
-  /**
-   * Check if the connection is alive
-   * @return true if alive
-   */
-  public boolean isAlive() {
-    return channel.isOpen();
-  }
-
-  public InetSocketAddress getAddress() {
-    return this.address;
-  }
-
-  /**
-   * Check if user should authenticate over Kerberos
-   * @return true if should be authenticated over Kerberos
-   * @throws java.io.IOException on failure of check
-   */
-  private synchronized boolean shouldAuthenticateOverKrb() throws IOException {
-    UserGroupInformation loginUser = UserGroupInformation.getLoginUser();
-    UserGroupInformation currentUser = UserGroupInformation.getCurrentUser();
-    UserGroupInformation realUser = currentUser.getRealUser();
-    return authMethod == AuthMethod.KERBEROS && loginUser != null &&
-      // Make sure user logged in using Kerberos either keytab or TGT
-      loginUser.hasKerberosCredentials() &&
-      // relogin only in case it is the login user (e.g. JT)
-      // or superuser (like oozie).
-      (loginUser.equals(currentUser) || loginUser.equals(realUser));
-  }
-
-  /**
-   * If multiple clients with the same principal try to connect to the same server at the same time,
-   * the server assumes a replay attack is in progress. This is a feature of kerberos. In order to
-   * work around this, what is done is that the client backs off randomly and tries to initiate the
-   * connection again. The other problem is to do with ticket expiry. To handle that, a relogin is
-   * attempted.
-   * <p>
-   * The retry logic is governed by the {@link #shouldAuthenticateOverKrb} method. In case when the
-   * user doesn't have valid credentials, we don't need to retry (from cache or ticket). In such
-   * cases, it is prudent to throw a runtime exception when we receive a SaslException from the
-   * underlying authentication implementation, so there is no retry from other high level (for eg,
-   * HCM or HBaseAdmin).
-   * </p>
-   * @param currRetries retry count
-   * @param ex exception describing fail
-   * @param user which is trying to connect
-   * @throws java.io.IOException if IO fail
-   * @throws InterruptedException if thread is interrupted
-   */
-  private void handleSaslConnectionFailure(final int currRetries, final Throwable ex,
-      final UserGroupInformation user) throws IOException, InterruptedException {
-    user.doAs(new PrivilegedExceptionAction<Void>() {
-      @Override
-      public Void run() throws IOException, InterruptedException {
-        if (shouldAuthenticateOverKrb()) {
-          if (currRetries < MAX_SASL_RETRIES) {
-            LOG.debug("Exception encountered while connecting to the server : " + ex);
-            // try re-login
-            if (UserGroupInformation.isLoginKeytabBased()) {
-              UserGroupInformation.getLoginUser().reloginFromKeytab();
-            } else {
-              UserGroupInformation.getLoginUser().reloginFromTicketCache();
-            }
-
-            // Should reconnect
-            return null;
-          } else {
-            String msg = "Couldn't setup connection for "
-                + UserGroupInformation.getLoginUser().getUserName() + " to " + serverPrincipal;
-            LOG.warn(msg, ex);
-            throw (IOException) new IOException(msg).initCause(ex);
-          }
-        } else {
-          LOG.warn("Exception encountered while connecting to " + "the server : " + ex);
-        }
-        if (ex instanceof RemoteException) {
-          throw (RemoteException) ex;
-        }
-        if (ex instanceof SaslException) {
-          String msg = "SASL authentication failed."
-              + " The most likely cause is missing or invalid credentials." + " Consider 'kinit'.";
-          LOG.fatal(msg, ex);
-          throw new RuntimeException(msg, ex);
-        }
-        throw new IOException(ex);
-      }
-    });
-  }
-
-  public int getConnectionHashCode() {
-    return ConnectionId.hashCode(ticket, serviceName, address);
-  }
-
-  @Override
-  public int hashCode() {
-    return getConnectionHashCode();
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (obj instanceof AsyncRpcChannel) {
-      AsyncRpcChannel channel = (AsyncRpcChannel) obj;
-      return channel.hashCode() == obj.hashCode();
-    }
-    return false;
-  }
-
-  @Override
-  public String toString() {
-    return this.address.toString() + "/" + this.serviceName + "/" + this.ticket;
-  }
-
-  /**
-   * Listens to call writes and fails if write failed
-   */
-  private static final class CallWriteListener implements ChannelFutureListener {
-    private final AsyncRpcChannel rpcChannel;
-    private final int id;
-
-    public CallWriteListener(AsyncRpcChannel asyncRpcChannelImpl, int id) {
-      this.rpcChannel = asyncRpcChannelImpl;
-      this.id = id;
-    }
-
-    @Override
-    public void operationComplete(ChannelFuture future) throws Exception {
-      if (!future.isSuccess()) {
-        AsyncCall call = rpcChannel.removePendingCall(id);
-        if (call != null) {
-          if (future.cause() instanceof IOException) {
-            call.setFailed((IOException) future.cause());
-          } else {
-            call.setFailed(new IOException(future.cause()));
-          }
-        }
-      }
-    }
-  }
-}


[3/6] hbase git commit: HBASE-16445 Refactor and reimplement RpcClient

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientImpl.java
deleted file mode 100644
index 4546c8d..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientImpl.java
+++ /dev/null
@@ -1,1359 +0,0 @@
-/**
- * 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.ipc;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.protobuf.Descriptors.MethodDescriptor;
-import com.google.protobuf.Message;
-import com.google.protobuf.Message.Builder;
-import com.google.protobuf.RpcCallback;
-import com.google.protobuf.RpcChannel;
-
-import java.io.BufferedInputStream;
-import java.io.BufferedOutputStream;
-import java.io.Closeable;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InterruptedIOException;
-import java.io.OutputStream;
-import java.net.InetSocketAddress;
-import java.net.Socket;
-import java.net.SocketAddress;
-import java.net.SocketTimeoutException;
-import java.net.UnknownHostException;
-import java.nio.ByteBuffer;
-import java.security.PrivilegedExceptionAction;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.Locale;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Random;
-import java.util.Set;
-import java.util.concurrent.ArrayBlockingQueue;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.ConcurrentSkipListMap;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import javax.net.SocketFactory;
-import javax.security.sasl.SaslException;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.CellScanner;
-import org.apache.hadoop.hbase.DoNotRetryIOException;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.client.MetricsConnection;
-import org.apache.hadoop.hbase.codec.Codec;
-import org.apache.hadoop.hbase.exceptions.ConnectionClosingException;
-import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos;
-import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta;
-import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ConnectionHeader;
-import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse;
-import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader;
-import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader;
-import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation;
-import org.apache.hadoop.hbase.protobuf.generated.TracingProtos.RPCTInfo;
-import org.apache.hadoop.hbase.security.AuthMethod;
-import org.apache.hadoop.hbase.security.HBaseSaslRpcClient;
-import org.apache.hadoop.hbase.security.SaslUtil.QualityOfProtection;
-import org.apache.hadoop.hbase.security.SecurityInfo;
-import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.security.UserProvider;
-import org.apache.hadoop.hbase.security.token.AuthenticationTokenSelector;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.ExceptionUtil;
-import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.hbase.util.PoolMap;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.SecurityUtil;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.security.token.TokenIdentifier;
-import org.apache.hadoop.security.token.TokenSelector;
-import org.apache.htrace.Span;
-import org.apache.htrace.Trace;
-import org.apache.htrace.TraceScope;
-
-/**
- * Does RPC against a cluster.  Manages connections per regionserver in the cluster.
- * <p>See HBaseServer
- */
-@InterfaceAudience.Private
-public class RpcClientImpl extends AbstractRpcClient {
-  private static final Log LOG = LogFactory.getLog(RpcClientImpl.class);
-  protected final AtomicInteger callIdCnt = new AtomicInteger();
-
-  protected final PoolMap<ConnectionId, Connection> connections;
-
-  protected final AtomicBoolean running = new AtomicBoolean(true); // if client runs
-
-  protected final FailedServers failedServers;
-
-  protected final SocketFactory socketFactory;           // how to create sockets
-
-  protected final static Map<AuthenticationProtos.TokenIdentifier.Kind,
-      TokenSelector<? extends TokenIdentifier>> tokenHandlers = new HashMap<>();
-  static {
-    tokenHandlers.put(AuthenticationProtos.TokenIdentifier.Kind.HBASE_AUTH_TOKEN,
-        new AuthenticationTokenSelector());
-  }
-
-  /**
-   * Creates a connection. Can be overridden by a subclass for testing.
-   * @param remoteId - the ConnectionId to use for the connection creation.
-   */
-  protected Connection createConnection(ConnectionId remoteId, final Codec codec,
-      final CompressionCodec compressor)
-  throws IOException {
-    return new Connection(remoteId, codec, compressor);
-  }
-
-  /**
-   * see {@link RpcClientImpl.Connection.CallSender}
-   */
-  private static class CallFuture {
-    final Call call;
-    final int priority;
-    final Span span;
-
-    // We will use this to stop the writer
-    final static CallFuture DEATH_PILL = new CallFuture(null, -1, null);
-
-    CallFuture(Call call, int priority, Span span) {
-      this.call = call;
-      this.priority = priority;
-      this.span = span;
-    }
-  }
-
-  /** Thread that reads responses and notifies callers.  Each connection owns a
-   * socket connected to a remote address.  Calls are multiplexed through this
-   * socket: responses may be delivered out of order. */
-  protected class Connection extends Thread {
-    private final ConnectionHeader header;              // connection header
-    protected ConnectionId remoteId;
-    protected Socket socket = null;                 // connected socket
-    protected DataInputStream in;
-    protected DataOutputStream out;
-    private final Object outLock = new Object();
-    private final InetSocketAddress server;             // server ip:port
-    private String serverPrincipal;  // server's krb5 principal name
-    private AuthMethod authMethod; // authentication method
-    private boolean useSasl;
-    private Token<? extends TokenIdentifier> token;
-    private HBaseSaslRpcClient saslRpcClient;
-    private final int reloginMaxBackoff; // max pause before relogin on sasl failure
-    private final Codec codec;
-    private final CompressionCodec compressor;
-
-    // currently active calls
-    protected final ConcurrentSkipListMap<Integer, Call> calls =
-      new ConcurrentSkipListMap<>();
-
-    protected final AtomicBoolean shouldCloseConnection = new AtomicBoolean();
-    protected final CallSender callSender;
-
-
-    /**
-     * If the client wants to interrupt its calls easily (i.e. call Thread#interrupt),
-     *  it gets into a java issue: an interruption during a write closes the socket/channel.
-     * A way to avoid this is to use a different thread for writing. This way, on interruptions,
-     *  we either cancel the writes or ignore the answer if the write is already done, but we
-     *  don't stop the write in the middle.
-     * This adds a thread per region server in the client, so it's kept as an option.
-     * <p>
-     * The implementation is simple: the client threads adds their call to the queue, and then
-     *  wait for an answer. The CallSender blocks on the queue, and writes the calls one
-     *  after the other. On interruption, the client cancels its call. The CallSender checks that
-     *  the call has not been canceled before writing it.
-     * </p>
-     * When the connection closes, all the calls not yet sent are dismissed. The client thread
-     *  is notified with an appropriate exception, as if the call was already sent but the answer
-     *  not yet received.
-     * </p>
-     */
-    private class CallSender extends Thread implements Closeable {
-      protected final BlockingQueue<CallFuture> callsToWrite;
-
-
-      public CallFuture sendCall(Call call, int priority, Span span)
-          throws InterruptedException, IOException {
-        CallFuture cts = new CallFuture(call, priority, span);
-        if (!callsToWrite.offer(cts)) {
-          throw new IOException("Can't add the call " + call.id +
-              " to the write queue. callsToWrite.size()=" + callsToWrite.size());
-        }
-        checkIsOpen(); // We check after the put, to be sure that the call we added won't stay
-                       //  in the list while the cleanup was already done.
-        return cts;
-      }
-
-      @Override
-      public void close(){
-        assert shouldCloseConnection.get();
-        callsToWrite.offer(CallFuture.DEATH_PILL);
-        // We don't care if we can't add the death pill to the queue: the writer
-        //  won't be blocked in the 'take', as its queue is full.
-      }
-
-      CallSender(String name, Configuration conf) {
-        int queueSize = conf.getInt("hbase.ipc.client.write.queueSize", 1000);
-        callsToWrite = new ArrayBlockingQueue<>(queueSize);
-        setDaemon(true);
-        setName(name + " - writer");
-      }
-
-      public void remove(CallFuture cts){
-        callsToWrite.remove(cts);
-
-        // By removing the call from the expected call list, we make the list smaller, but
-        //  it means as well that we don't know how many calls we cancelled.
-        calls.remove(cts.call.id);
-        cts.call.callComplete();
-      }
-
-      /**
-       * Reads the call from the queue, write them on the socket.
-       */
-      @Override
-      public void run() {
-        while (!shouldCloseConnection.get()) {
-          CallFuture cts = null;
-          try {
-            cts = callsToWrite.take();
-          } catch (InterruptedException e) {
-            markClosed(new InterruptedIOException());
-          }
-
-          if (cts == null || cts == CallFuture.DEATH_PILL) {
-            assert shouldCloseConnection.get();
-            break;
-          }
-
-          if (cts.call.done) {
-            continue;
-          }
-
-          if (cts.call.checkAndSetTimeout()) {
-            continue;
-          }
-
-          try {
-            Connection.this.tracedWriteRequest(cts.call, cts.priority, cts.span);
-          } catch (IOException e) {
-            if (LOG.isDebugEnabled()) {
-              LOG.debug("call write error for call #" + cts.call.id
-                + ", message =" + e.getMessage());
-            }
-            cts.call.setException(e);
-            markClosed(e);
-          }
-        }
-
-        cleanup();
-      }
-
-      /**
-       * Cleans the call not yet sent when we finish.
-       */
-      private void cleanup() {
-        assert shouldCloseConnection.get();
-
-        IOException ie = new ConnectionClosingException("Connection to " + server + " is closing.");
-        while (true) {
-          CallFuture cts = callsToWrite.poll();
-          if (cts == null) {
-            break;
-          }
-          if (cts.call != null && !cts.call.done) {
-            cts.call.setException(ie);
-          }
-        }
-      }
-    }
-
-    Connection(ConnectionId remoteId, final Codec codec, final CompressionCodec compressor)
-    throws IOException {
-      if (remoteId.getAddress().isUnresolved()) {
-        throw new UnknownHostException("unknown host: " + remoteId.getAddress().getHostName());
-      }
-      this.server = remoteId.getAddress();
-      this.codec = codec;
-      this.compressor = compressor;
-
-      UserGroupInformation ticket = remoteId.getTicket().getUGI();
-      SecurityInfo securityInfo = SecurityInfo.getInfo(remoteId.getServiceName());
-      this.useSasl = userProvider.isHBaseSecurityEnabled();
-      if (useSasl && securityInfo != null) {
-        AuthenticationProtos.TokenIdentifier.Kind tokenKind = securityInfo.getTokenKind();
-        if (tokenKind != null) {
-          TokenSelector<? extends TokenIdentifier> tokenSelector =
-              tokenHandlers.get(tokenKind);
-          if (tokenSelector != null) {
-            token = tokenSelector.selectToken(new Text(clusterId),
-                ticket.getTokens());
-          } else if (LOG.isDebugEnabled()) {
-            LOG.debug("No token selector found for type "+tokenKind);
-          }
-        }
-        String serverKey = securityInfo.getServerPrincipal();
-        if (serverKey == null) {
-          throw new IOException(
-              "Can't obtain server Kerberos config key from SecurityInfo");
-        }
-        serverPrincipal = SecurityUtil.getServerPrincipal(
-            conf.get(serverKey), server.getAddress().getCanonicalHostName().toLowerCase(Locale.ROOT));
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("RPC Server Kerberos principal name for service="
-              + remoteId.getServiceName() + " is " + serverPrincipal);
-        }
-      }
-
-      if (!useSasl) {
-        authMethod = AuthMethod.SIMPLE;
-      } else if (token != null) {
-        authMethod = AuthMethod.DIGEST;
-      } else {
-        authMethod = AuthMethod.KERBEROS;
-      }
-
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Use " + authMethod + " authentication for service " + remoteId.serviceName +
-          ", sasl=" + useSasl);
-      }
-      reloginMaxBackoff = conf.getInt("hbase.security.relogin.maxbackoff", 5000);
-      this.remoteId = remoteId;
-
-      ConnectionHeader.Builder builder = ConnectionHeader.newBuilder();
-      builder.setServiceName(remoteId.getServiceName());
-      UserInformation userInfoPB = getUserInfo(ticket);
-      if (userInfoPB != null) {
-        builder.setUserInfo(userInfoPB);
-      }
-      if (this.codec != null) {
-        builder.setCellBlockCodecClass(this.codec.getClass().getCanonicalName());
-      }
-      if (this.compressor != null) {
-        builder.setCellBlockCompressorClass(this.compressor.getClass().getCanonicalName());
-      }
-      builder.setVersionInfo(ProtobufUtil.getVersionInfo());
-      this.header = builder.build();
-
-      this.setName("IPC Client (" + socketFactory.hashCode() +") connection to " +
-        remoteId.getAddress().toString() +
-        ((ticket==null)?" from an unknown user": (" from "
-        + ticket.getUserName())));
-      this.setDaemon(true);
-
-      if (conf.getBoolean(SPECIFIC_WRITE_THREAD, false)) {
-        callSender = new CallSender(getName(), conf);
-        callSender.start();
-      } else {
-        callSender = null;
-      }
-    }
-
-    private synchronized UserInformation getUserInfo(UserGroupInformation ugi) {
-      if (ugi == null || authMethod == AuthMethod.DIGEST) {
-        // Don't send user for token auth
-        return null;
-      }
-      UserInformation.Builder userInfoPB = UserInformation.newBuilder();
-      if (authMethod == AuthMethod.KERBEROS) {
-        // Send effective user for Kerberos auth
-        userInfoPB.setEffectiveUser(ugi.getUserName());
-      } else if (authMethod == AuthMethod.SIMPLE) {
-        //Send both effective user and real user for simple auth
-        userInfoPB.setEffectiveUser(ugi.getUserName());
-        if (ugi.getRealUser() != null) {
-          userInfoPB.setRealUser(ugi.getRealUser().getUserName());
-        }
-      }
-      return userInfoPB.build();
-    }
-
-    protected synchronized void setupConnection() throws IOException {
-      short ioFailures = 0;
-      short timeoutFailures = 0;
-      while (true) {
-        try {
-          this.socket = socketFactory.createSocket();
-          this.socket.setTcpNoDelay(tcpNoDelay);
-          this.socket.setKeepAlive(tcpKeepAlive);
-          if (localAddr != null) {
-            this.socket.bind(localAddr);
-          }
-          NetUtils.connect(this.socket, remoteId.getAddress(), connectTO);
-          this.socket.setSoTimeout(readTO);
-          return;
-        } catch (SocketTimeoutException toe) {
-          /* The max number of retries is 45,
-           * which amounts to 20s*45 = 15 minutes retries.
-           */
-          handleConnectionFailure(timeoutFailures++, maxRetries, toe);
-        } catch (IOException ie) {
-          handleConnectionFailure(ioFailures++, maxRetries, ie);
-        }
-      }
-    }
-
-    protected synchronized void closeConnection() {
-      if (socket == null) {
-        return;
-      }
-
-      // close the current connection
-      try {
-        if (socket.getOutputStream() != null) {
-          socket.getOutputStream().close();
-        }
-      } catch (IOException ignored) {  // Can happen if the socket is already closed
-        if (LOG.isTraceEnabled()){
-          LOG.trace("ignored", ignored);
-        }
-      }
-      try {
-        if (socket.getInputStream() != null) {
-          socket.getInputStream().close();
-        }
-      } catch (IOException ignored) {  // Can happen if the socket is already closed
-        if (LOG.isTraceEnabled()){
-          LOG.trace("ignored", ignored);
-        }
-      }
-      try {
-        if (socket.getChannel() != null) {
-          socket.getChannel().close();
-        }
-      } catch (IOException ignored) {  // Can happen if the socket is already closed
-        if (LOG.isTraceEnabled()){
-          LOG.trace("ignored", ignored);
-        }
-      }
-      try {
-        socket.close();
-      } catch (IOException e) {
-        LOG.warn("Not able to close a socket", e);
-      }
-
-      // set socket to null so that the next call to setupIOstreams
-      // can start the process of connect all over again.
-      socket = null;
-    }
-
-    /**
-     *  Handle connection failures
-     *
-     * If the current number of retries is equal to the max number of retries,
-     * stop retrying and throw the exception; Otherwise backoff N seconds and
-     * try connecting again.
-     *
-     * This Method is only called from inside setupIOstreams(), which is
-     * synchronized. Hence the sleep is synchronized; the locks will be retained.
-     *
-     * @param curRetries current number of retries
-     * @param maxRetries max number of retries allowed
-     * @param ioe failure reason
-     * @throws IOException if max number of retries is reached
-     */
-    private void handleConnectionFailure(int curRetries, int maxRetries, IOException ioe)
-    throws IOException {
-      closeConnection();
-
-      // throw the exception if the maximum number of retries is reached
-      if (curRetries >= maxRetries || ExceptionUtil.isInterrupt(ioe)) {
-        throw ioe;
-      }
-
-      // otherwise back off and retry
-      try {
-        Thread.sleep(failureSleep);
-      } catch (InterruptedException ie) {
-        ExceptionUtil.rethrowIfInterrupt(ie);
-      }
-
-      LOG.info("Retrying connect to server: " + remoteId.getAddress() +
-        " after sleeping " + failureSleep + "ms. Already tried " + curRetries +
-        " time(s).");
-    }
-
-    /**
-     * @throws IOException if the connection is not open.
-     */
-    private void checkIsOpen() throws IOException {
-      if (shouldCloseConnection.get()) {
-        throw new ConnectionClosingException(getName() + " is closing");
-      }
-    }
-
-    /* wait till someone signals us to start reading RPC response or
-     * it is idle too long, it is marked as to be closed,
-     * or the client is marked as not running.
-     *
-     * @return true if it is time to read a response; false otherwise.
-     */
-    protected synchronized boolean waitForWork() throws InterruptedException {
-      // beware of the concurrent access to the calls list: we can add calls, but as well
-      //  remove them.
-      long waitUntil = EnvironmentEdgeManager.currentTime() + minIdleTimeBeforeClose;
-
-      while (true) {
-        if (shouldCloseConnection.get()) {
-          return false;
-        }
-
-        if (!running.get()) {
-          markClosed(new IOException("stopped with " + calls.size() + " pending request(s)"));
-          return false;
-        }
-
-        if (!calls.isEmpty()) {
-          // shouldCloseConnection can be set to true by a parallel thread here. The caller
-          //  will need to check anyway.
-          return true;
-        }
-
-        if (EnvironmentEdgeManager.currentTime() >= waitUntil) {
-          // Connection is idle.
-          // We expect the number of calls to be zero here, but actually someone can
-          //  adds a call at the any moment, as there is no synchronization between this task
-          //  and adding new calls. It's not a big issue, but it will get an exception.
-          markClosed(new IOException(
-              "idle connection closed with " + calls.size() + " pending request(s)"));
-          return false;
-        }
-
-        wait(Math.min(minIdleTimeBeforeClose, 1000));
-      }
-    }
-
-    public InetSocketAddress getRemoteAddress() {
-      return remoteId.getAddress();
-    }
-
-    @Override
-    public void run() {
-      if (LOG.isTraceEnabled()) {
-        LOG.trace(getName() + ": starting, connections " + connections.size());
-      }
-
-      try {
-        while (waitForWork()) { // Wait here for work - read or close connection
-          readResponse();
-        }
-      } catch (InterruptedException t) {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace(getName() + ": interrupted while waiting for call responses");
-        }
-        markClosed(ExceptionUtil.asInterrupt(t));
-      } catch (Throwable t) {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug(getName() + ": unexpected throwable while waiting for call responses", t);
-        }
-        markClosed(new IOException("Unexpected throwable while waiting call responses", t));
-      }
-
-      close();
-
-      if (LOG.isTraceEnabled()) {
-        LOG.trace(getName() + ": stopped, connections " + connections.size());
-      }
-    }
-
-    private synchronized void disposeSasl() {
-      if (saslRpcClient != null) {
-        try {
-          saslRpcClient.dispose();
-          saslRpcClient = null;
-        } catch (IOException ioe) {
-          LOG.error("Error disposing of SASL client", ioe);
-        }
-      }
-    }
-
-    private synchronized boolean shouldAuthenticateOverKrb() throws IOException {
-      UserGroupInformation loginUser = UserGroupInformation.getLoginUser();
-      UserGroupInformation currentUser =
-        UserGroupInformation.getCurrentUser();
-      UserGroupInformation realUser = currentUser.getRealUser();
-      return authMethod == AuthMethod.KERBEROS &&
-          loginUser != null &&
-          //Make sure user logged in using Kerberos either keytab or TGT
-          loginUser.hasKerberosCredentials() &&
-          // relogin only in case it is the login user (e.g. JT)
-          // or superuser (like oozie).
-          (loginUser.equals(currentUser) || loginUser.equals(realUser));
-    }
-
-    private synchronized boolean setupSaslConnection(final InputStream in2,
-        final OutputStream out2) throws IOException {
-      saslRpcClient = new HBaseSaslRpcClient(authMethod, token, serverPrincipal, fallbackAllowed,
-          conf.get("hbase.rpc.protection",
-              QualityOfProtection.AUTHENTICATION.name().toLowerCase(Locale.ROOT)));
-      return saslRpcClient.saslConnect(in2, out2);
-    }
-
-    /**
-     * If multiple clients with the same principal try to connect
-     * to the same server at the same time, the server assumes a
-     * replay attack is in progress. This is a feature of kerberos.
-     * In order to work around this, what is done is that the client
-     * backs off randomly and tries to initiate the connection
-     * again.
-     * The other problem is to do with ticket expiry. To handle that,
-     * a relogin is attempted.
-     * <p>
-     * The retry logic is governed by the {@link #shouldAuthenticateOverKrb}
-     * method. In case when the user doesn't have valid credentials, we don't
-     * need to retry (from cache or ticket). In such cases, it is prudent to
-     * throw a runtime exception when we receive a SaslException from the
-     * underlying authentication implementation, so there is no retry from
-     * other high level (for eg, HCM or HBaseAdmin).
-     * </p>
-     */
-    private synchronized void handleSaslConnectionFailure(
-        final int currRetries,
-        final int maxRetries, final Exception ex, final Random rand,
-        final UserGroupInformation user)
-    throws IOException, InterruptedException{
-      user.doAs(new PrivilegedExceptionAction<Object>() {
-        @Override
-        public Object run() throws IOException, InterruptedException {
-          closeConnection();
-          if (shouldAuthenticateOverKrb()) {
-            if (currRetries < maxRetries) {
-              if (LOG.isDebugEnabled()) {
-                LOG.debug("Exception encountered while connecting to " +
-                    "the server : " + ex);
-              }
-              //try re-login
-              if (UserGroupInformation.isLoginKeytabBased()) {
-                UserGroupInformation.getLoginUser().reloginFromKeytab();
-              } else {
-                UserGroupInformation.getLoginUser().reloginFromTicketCache();
-              }
-              disposeSasl();
-              //have granularity of milliseconds
-              //we are sleeping with the Connection lock held but since this
-              //connection instance is being used for connecting to the server
-              //in question, it is okay
-              Thread.sleep((rand.nextInt(reloginMaxBackoff) + 1));
-              return null;
-            } else {
-              String msg = "Couldn't setup connection for " +
-                UserGroupInformation.getLoginUser().getUserName() +
-                " to " + serverPrincipal;
-              LOG.warn(msg, ex);
-              throw (IOException) new IOException(msg).initCause(ex);
-            }
-          } else {
-            LOG.warn("Exception encountered while connecting to " +
-                "the server : " + ex);
-          }
-          if (ex instanceof RemoteException) {
-            throw (RemoteException)ex;
-          }
-          if (ex instanceof SaslException) {
-            String msg = "SASL authentication failed." +
-              " The most likely cause is missing or invalid credentials." +
-              " Consider 'kinit'.";
-            LOG.fatal(msg, ex);
-            throw new RuntimeException(msg, ex);
-          }
-          throw new IOException(ex);
-        }
-      });
-    }
-
-    protected synchronized void setupIOstreams() throws IOException {
-      if (socket != null) {
-        // The connection is already available. Perfect.
-        return;
-      }
-
-      if (shouldCloseConnection.get()){
-        throw new ConnectionClosingException("This connection is closing");
-      }
-
-      if (failedServers.isFailedServer(remoteId.getAddress())) {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Not trying to connect to " + server +
-              " this server is in the failed servers list");
-        }
-        IOException e = new FailedServerException(
-            "This server is in the failed servers list: " + server);
-        markClosed(e);
-        close();
-        throw e;
-      }
-
-      try {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Connecting to " + server);
-        }
-        short numRetries = 0;
-        final short MAX_RETRIES = 5;
-        Random rand = null;
-        while (true) {
-          setupConnection();
-          InputStream inStream = NetUtils.getInputStream(socket);
-          // This creates a socket with a write timeout. This timeout cannot be changed.
-          OutputStream outStream = NetUtils.getOutputStream(socket, writeTO);
-          // Write out the preamble -- MAGIC, version, and auth to use.
-          writeConnectionHeaderPreamble(outStream);
-          if (useSasl) {
-            final InputStream in2 = inStream;
-            final OutputStream out2 = outStream;
-            UserGroupInformation ticket = remoteId.getTicket().getUGI();
-            if (authMethod == AuthMethod.KERBEROS) {
-              if (ticket != null && ticket.getRealUser() != null) {
-                ticket = ticket.getRealUser();
-              }
-            }
-            boolean continueSasl;
-            if (ticket == null){
-              throw new FatalConnectionException("ticket/user is null");
-            }
-            try {
-              continueSasl = ticket.doAs(new PrivilegedExceptionAction<Boolean>() {
-                @Override
-                public Boolean run() throws IOException {
-                  return setupSaslConnection(in2, out2);
-                }
-              });
-            } catch (Exception ex) {
-              ExceptionUtil.rethrowIfInterrupt(ex);
-              if (rand == null) {
-                rand = new Random();
-              }
-              handleSaslConnectionFailure(numRetries++, MAX_RETRIES, ex, rand, ticket);
-              continue;
-            }
-            if (continueSasl) {
-              // Sasl connect is successful. Let's set up Sasl i/o streams.
-              inStream = saslRpcClient.getInputStream(inStream);
-              outStream = saslRpcClient.getOutputStream(outStream);
-            } else {
-              // fall back to simple auth because server told us so.
-              authMethod = AuthMethod.SIMPLE;
-              useSasl = false;
-            }
-          }
-          this.in = new DataInputStream(new BufferedInputStream(inStream));
-          synchronized (this.outLock) {
-            this.out = new DataOutputStream(new BufferedOutputStream(outStream));
-          }
-          // Now write out the connection header
-          writeConnectionHeader();
-
-          // start the receiver thread after the socket connection has been set up
-          start();
-          return;
-        }
-      } catch (Throwable t) {
-        IOException e = ExceptionUtil.asInterrupt(t);
-        if (e == null) {
-          failedServers.addToFailedServers(remoteId.address);
-          if (t instanceof LinkageError) {
-            // probably the hbase hadoop version does not match the running hadoop version
-            e = new DoNotRetryIOException(t);
-          } else if (t instanceof IOException) {
-            e = (IOException) t;
-          } else {
-            e = new IOException("Could not set up IO Streams to " + server, t);
-          }
-        }
-        markClosed(e);
-        close();
-        throw e;
-      }
-    }
-
-    /**
-     * Write the RPC header: {@code <MAGIC WORD -- 'HBas'> <ONEBYTE_VERSION> <ONEBYTE_AUTH_TYPE>}
-     */
-    private void writeConnectionHeaderPreamble(OutputStream outStream) throws IOException {
-      // Assemble the preamble up in a buffer first and then send it.  Writing individual elements,
-      // they are getting sent across piecemeal according to wireshark and then server is messing
-      // up the reading on occasion (the passed in stream is not buffered yet).
-
-      // Preamble is six bytes -- 'HBas' + VERSION + AUTH_CODE
-      int rpcHeaderLen = HConstants.RPC_HEADER.length;
-      byte [] preamble = new byte [rpcHeaderLen + 2];
-      System.arraycopy(HConstants.RPC_HEADER, 0, preamble, 0, rpcHeaderLen);
-      preamble[rpcHeaderLen] = HConstants.RPC_CURRENT_VERSION;
-      synchronized (this) {
-        preamble[rpcHeaderLen + 1] = authMethod.code;
-      }
-      outStream.write(preamble);
-      outStream.flush();
-    }
-
-    /**
-     * Write the connection header.
-     */
-    private synchronized void writeConnectionHeader() throws IOException {
-      synchronized (this.outLock) {
-        this.out.writeInt(this.header.getSerializedSize());
-        this.header.writeTo(this.out);
-        this.out.flush();
-      }
-    }
-
-    /** Close the connection. */
-    protected synchronized void close() {
-      if (!shouldCloseConnection.get()) {
-        LOG.error(getName() + ": the connection is not in the closed state");
-        return;
-      }
-
-      // release the resources
-      // first thing to do;take the connection out of the connection list
-      synchronized (connections) {
-        connections.removeValue(remoteId, this);
-      }
-
-      // close the streams and therefore the socket
-      synchronized(this.outLock) {
-        if (this.out != null) {
-          IOUtils.closeStream(out);
-          this.out = null;
-        }
-      }
-      IOUtils.closeStream(in);
-      this.in = null;
-      if (this.socket != null) {
-        try {
-          this.socket.close();
-          this.socket = null;
-        } catch (IOException e) {
-          LOG.error("Error while closing socket", e);
-        }
-      }
-
-      disposeSasl();
-
-      // log the info
-      if (LOG.isTraceEnabled()) {
-        LOG.trace(getName() + ": closing ipc connection to " + server);
-      }
-
-      cleanupCalls(true);
-
-      if (LOG.isTraceEnabled()) {
-        LOG.trace(getName() + ": ipc connection to " + server + " closed");
-      }
-    }
-
-    protected void tracedWriteRequest(Call call, int priority, Span span) throws IOException {
-      try (TraceScope ignored = Trace.startSpan("RpcClientImpl.tracedWriteRequest", span)) {
-        writeRequest(call, priority, span);
-      }
-    }
-
-    /**
-     * Initiates a call by sending the parameter to the remote server.
-     * Note: this is not called from the Connection thread, but by other
-     * threads.
-     * @see #readResponse()
-     */
-    @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="IS2_INCONSISTENT_SYNC",
-        justification="Findbugs is misinterpreting locking missing fact that this.outLock is held")
-    private void writeRequest(Call call, final int priority, Span span) throws IOException {
-      RequestHeader.Builder builder = RequestHeader.newBuilder();
-      builder.setCallId(call.id);
-      if (span != null) {
-        builder.setTraceInfo(
-            RPCTInfo.newBuilder().setParentId(span.getSpanId()).setTraceId(span.getTraceId()));
-      }
-      builder.setMethodName(call.md.getName());
-      builder.setRequestParam(call.param != null);
-      ByteBuffer cellBlock = cellBlockBuilder.buildCellBlock(this.codec, this.compressor, call.cells);
-      if (cellBlock != null) {
-        CellBlockMeta.Builder cellBlockBuilder = CellBlockMeta.newBuilder();
-        cellBlockBuilder.setLength(cellBlock.limit());
-        builder.setCellBlockMeta(cellBlockBuilder.build());
-      }
-      // Only pass priority if there is one set.
-      if (priority != HBaseRpcController.PRIORITY_UNSET) {
-        builder.setPriority(priority);
-      }
-      builder.setTimeout(call.timeout);
-      RequestHeader requestHeader = builder.build();
-
-      setupIOstreams();
-
-      // Now we're going to write the call. We take the lock, then check that the connection
-      //  is still valid, and, if so we do the write to the socket. If the write fails, we don't
-      //  know where we stand, we have to close the connection.
-      checkIsOpen();
-      IOException writeException = null;
-      synchronized (this.outLock) {
-        if (Thread.interrupted()){
-          throw new InterruptedIOException();
-        }
-
-        calls.put(call.id, call); // We put first as we don't want the connection to become idle.
-        checkIsOpen(); // Now we're checking that it didn't became idle in between.
-
-        try {
-          call.callStats.setRequestSizeBytes(IPCUtil.write(this.out, requestHeader, call.param,
-              cellBlock));
-        } catch (IOException e) {
-          // We set the value inside the synchronized block, this way the next in line
-          //  won't even try to write. Otherwise we might miss a call in the calls map?
-          shouldCloseConnection.set(true);
-          writeException = e;
-          interrupt();
-        }
-      }
-
-      // call close outside of the synchronized (outLock) to prevent deadlock - HBASE-14474
-      if (writeException != null) {
-        markClosed(writeException);
-        close();
-      }
-
-      // We added a call, and may be started the connection close. In both cases, we
-      //  need to notify the reader.
-      doNotify();
-
-      // Now that we notified, we can rethrow the exception if any. Otherwise we're good.
-      if (writeException != null){
-        throw writeException;
-      }
-    }
-
-    @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NN_NAKED_NOTIFY",
-        justification="Presume notifyAll is because we are closing/shutting down")
-    private synchronized void doNotify() {
-      // Make a separate method so can do synchronize and add findbugs annotation; only one
-      // annotation at at time in source 1.7.
-      notifyAll(); // Findbugs: NN_NAKED_NOTIFY
-    }
-
-    /* Receive a response.
-     * Because only one receiver, so no synchronization on in.
-     */
-    protected void readResponse() {
-      if (shouldCloseConnection.get()){
-        return;
-      }
-      Call call = null;
-      boolean expectedCall = false;
-      try {
-        // See HBaseServer.Call.setResponse for where we write out the response.
-        // Total size of the response.  Unused.  But have to read it in anyways.
-        int totalSize = in.readInt();
-
-        // Read the header
-        ResponseHeader responseHeader = ResponseHeader.parseDelimitedFrom(in);
-        int id = responseHeader.getCallId();
-        call = calls.remove(id); // call.done have to be set before leaving this method
-        expectedCall = (call != null && !call.done);
-        if (!expectedCall) {
-          // So we got a response for which we have no corresponding 'call' here on the client-side.
-          // We probably timed out waiting, cleaned up all references, and now the server decides
-          // to return a response.  There is nothing we can do w/ the response at this stage. Clean
-          // out the wire of the response so its out of the way and we can get other responses on
-          // this connection.
-          int readSoFar = IPCUtil.getTotalSizeWhenWrittenDelimited(responseHeader);
-          int whatIsLeftToRead = totalSize - readSoFar;
-          IOUtils.skipFully(in, whatIsLeftToRead);
-          if (call != null) {
-            call.callStats.setResponseSizeBytes(totalSize);
-            call.callStats.setCallTimeMs(
-                EnvironmentEdgeManager.currentTime() - call.callStats.getStartTime());
-          }
-          return;
-        }
-        if (responseHeader.hasException()) {
-          ExceptionResponse exceptionResponse = responseHeader.getException();
-          RemoteException re = IPCUtil.createRemoteException(exceptionResponse);
-          call.setException(re);
-          call.callStats.setResponseSizeBytes(totalSize);
-          call.callStats.setCallTimeMs(
-              EnvironmentEdgeManager.currentTime() - call.callStats.getStartTime());
-          if (IPCUtil.isFatalConnectionException(exceptionResponse)) {
-            markClosed(re);
-          }
-        } else {
-          Message value = null;
-          if (call.responseDefaultType != null) {
-            Builder builder = call.responseDefaultType.newBuilderForType();
-            ProtobufUtil.mergeDelimitedFrom(builder, in);
-            value = builder.build();
-          }
-          CellScanner cellBlockScanner = null;
-          if (responseHeader.hasCellBlockMeta()) {
-            int size = responseHeader.getCellBlockMeta().getLength();
-            byte [] cellBlock = new byte[size];
-            IOUtils.readFully(this.in, cellBlock, 0, cellBlock.length);
-            cellBlockScanner = cellBlockBuilder.createCellScanner(this.codec, this.compressor, cellBlock);
-          }
-          call.setResponse(value, cellBlockScanner);
-          call.callStats.setResponseSizeBytes(totalSize);
-          call.callStats.setCallTimeMs(
-              EnvironmentEdgeManager.currentTime() - call.callStats.getStartTime());
-        }
-      } catch (IOException e) {
-        if (expectedCall){
-          call.setException(e);
-        }
-        if (e instanceof SocketTimeoutException) {
-          // Clean up open calls but don't treat this as a fatal condition,
-          // since we expect certain responses to not make it by the specified
-          // {@link ConnectionId#rpcTimeout}.
-          if (LOG.isTraceEnabled()){
-            LOG.trace("ignored", e);
-          }
-        } else {
-          // Treat this as a fatal condition and close this connection
-          markClosed(e);
-        }
-      } finally {
-        cleanupCalls(false);
-      }
-    }
-
-    protected synchronized boolean markClosed(IOException e) {
-      if (e == null){
-        throw new NullPointerException();
-      }
-
-      boolean ret = shouldCloseConnection.compareAndSet(false, true);
-      if (ret) {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace(getName() + ": marking at should close, reason: " + e.getMessage());
-        }
-        if (callSender != null) {
-          callSender.close();
-        }
-        notifyAll();
-      }
-      return ret;
-    }
-
-
-    /**
-     * Cleanup the calls older than a given timeout, in milli seconds.
-     * @param allCalls true for all calls, false for only the calls in timeout
-     */
-    protected synchronized void cleanupCalls(boolean allCalls) {
-      Iterator<Entry<Integer, Call>> itor = calls.entrySet().iterator();
-      while (itor.hasNext()) {
-        Call c = itor.next().getValue();
-        if (c.done) {
-          // To catch the calls without timeout that were cancelled.
-          itor.remove();
-        } else if (allCalls) {
-          long waitTime = EnvironmentEdgeManager.currentTime() - c.getStartTime();
-          IOException ie = new ConnectionClosingException("Connection to " + getRemoteAddress()
-              + " is closing. Call id=" + c.id + ", waitTime=" + waitTime);
-          c.setException(ie);
-          itor.remove();
-        } else if (c.checkAndSetTimeout()) {
-          itor.remove();
-        } else {
-          // We expect the call to be ordered by timeout. It may not be the case, but stopping
-          //  at the first valid call allows to be sure that we still have something to do without
-          //  spending too much time by reading the full list.
-          break;
-        }
-      }
-    }
-  }
-
-  /**
-   * Used in test only. Construct an IPC cluster client whose values are of the
-   * {@link Message} class.
-   * @param conf configuration
-   * @param clusterId the cluster id
-   * @param factory socket factory
-   */
-  @VisibleForTesting
-  RpcClientImpl(Configuration conf, String clusterId, SocketFactory factory) {
-    this(conf, clusterId, factory, null, null);
-  }
-
-  /**
-   * Construct an IPC cluster client whose values are of the {@link Message} class.
-   * @param conf configuration
-   * @param clusterId the cluster id
-   * @param factory socket factory
-   * @param localAddr client socket bind address
-   * @param metrics the connection metrics
-   */
-  RpcClientImpl(Configuration conf, String clusterId, SocketFactory factory,
-      SocketAddress localAddr, MetricsConnection metrics) {
-    super(conf, clusterId, localAddr, metrics);
-
-    this.socketFactory = factory;
-    this.connections = new PoolMap<>(getPoolType(conf), getPoolSize(conf));
-    this.failedServers = new FailedServers(conf);
-  }
-
-  /**
-   * Used in test only. Construct an IPC client for the cluster {@code clusterId} with
-   * the default SocketFactory
-   */
-  @VisibleForTesting
-  RpcClientImpl(Configuration conf, String clusterId) {
-    this(conf, clusterId, NetUtils.getDefaultSocketFactory(conf), null, null);
-  }
-
-  /**
-   * Construct an IPC client for the cluster {@code clusterId} with the default SocketFactory
-   *
-   * This method is called with reflection by the RpcClientFactory to create an instance
-   *
-   * @param conf configuration
-   * @param clusterId the cluster id
-   * @param localAddr client socket bind address.
-   * @param metrics the connection metrics
-   */
-  public RpcClientImpl(Configuration conf, String clusterId, SocketAddress localAddr,
-      MetricsConnection metrics) {
-    this(conf, clusterId, NetUtils.getDefaultSocketFactory(conf), localAddr, metrics);
-  }
-
-  /** Stop all threads related to this client.  No further calls may be made
-   * using this client. */
-  @Override
-  public void close() {
-    if (LOG.isDebugEnabled()){
-      LOG.debug("Stopping rpc client");
-    }
-    if (!running.compareAndSet(true, false)){
-      return;
-    }
-
-    Set<Connection> connsToClose = null;
-    // wake up all connections
-    synchronized (connections) {
-      for (Connection conn : connections.values()) {
-        conn.interrupt();
-        if (conn.callSender != null) {
-          conn.callSender.interrupt();
-        }
-
-        // In case the CallSender did not setupIOStreams() yet, the Connection may not be started
-        // at all (if CallSender has a cancelled Call it can happen). See HBASE-13851
-        if (!conn.isAlive()) {
-          if (connsToClose == null) {
-            connsToClose = new HashSet<>();
-          }
-          connsToClose.add(conn);
-        }
-      }
-    }
-    if (connsToClose != null) {
-      for (Connection conn : connsToClose) {
-        conn.markClosed(new InterruptedIOException("RpcClient is closing"));
-        conn.close();
-      }
-    }
-    // wait until all connections are closed
-    while (!connections.isEmpty()) {
-      try {
-        Thread.sleep(10);
-      } catch (InterruptedException e) {
-        LOG.info("Interrupted while stopping the client. We still have " + connections.size() +
-            " connections.");
-        Thread.currentThread().interrupt();
-        return;
-      }
-    }
-  }
-
-  /**
-   * Make a call, passing <code>param</code>, to the IPC server running at
-   * <code>address</code> which is servicing the <code>protocol</code> protocol,
-   * with the <code>ticket</code> credentials, returning the value.
-   * Throws exceptions if there are network problems or if the remote code
-   * threw an exception.
-   * @param ticket Be careful which ticket you pass. A new user will mean a new Connection.
-   *          {@link UserProvider#getCurrent()} makes a new instance of User each time so will be a
-   *          new Connection each time.
-   * @return A pair with the Message response and the Cell data (if any).
-   * @throws InterruptedException if the call is interrupted
-   * @throws IOException if something fails on the connection
-   */
-  @Override
-  protected Pair<Message, CellScanner> call(HBaseRpcController pcrc, MethodDescriptor md,
-      Message param, Message returnType, User ticket, InetSocketAddress addr,
-      MetricsConnection.CallStats callStats)
-      throws IOException, InterruptedException {
-    if (pcrc == null) {
-      pcrc = new HBaseRpcControllerImpl();
-    }
-
-    Call call = this.call(md, param, returnType, pcrc, ticket, addr, callStats);
-
-    return new Pair<>(call.response, call.cells);
-  }
-
-
-  /**
-   * Make a call, passing <code>param</code>, to the IPC server running at
-   * <code>address</code> which is servicing the <code>protocol</code> protocol,
-   * with the <code>ticket</code> credentials, returning the value.
-   * Throws exceptions if there are network problems or if the remote code
-   * threw an exception.
-   * @param ticket Be careful which ticket you pass. A new user will mean a new Connection.
-   *          {@link UserProvider#getCurrent()} makes a new instance of User each time so will be a
-   *          new Connection each time.
-   * @return A Call
-   * @throws InterruptedException if the call is interrupted
-   * @throws IOException if something fails on the connection
-   */
-  private <R extends Message> Call call(MethodDescriptor method, Message request,
-      R responsePrototype, HBaseRpcController pcrc, User ticket,
-      InetSocketAddress addr, MetricsConnection.CallStats callStats)
-      throws IOException, InterruptedException {
-
-    CellScanner cells = pcrc.cellScanner();
-
-    final Call call = new Call(callIdCnt.getAndIncrement(), method, request, cells,
-        responsePrototype, pcrc.getCallTimeout(), callStats);
-
-    final Connection connection = getConnection(ticket, call, addr);
-
-    final CallFuture cts;
-    if (connection.callSender != null) {
-      cts = connection.callSender.sendCall(call, pcrc.getPriority(), Trace.currentSpan());
-      pcrc.notifyOnCancel(new RpcCallback<Object>() {
-        @Override
-        public void run(Object parameter) {
-          connection.callSender.remove(cts);
-        }
-      });
-      if (pcrc.isCanceled()) {
-        // To finish if the call was cancelled before we set the notification (race condition)
-        call.callComplete();
-        return call;
-      }
-    } else {
-      cts = null;
-      connection.tracedWriteRequest(call, pcrc.getPriority(), Trace.currentSpan());
-    }
-
-    while (!call.done) {
-      if (call.checkAndSetTimeout()) {
-        if (cts != null){
-          connection.callSender.remove(cts);
-        }
-        break;
-      }
-      if (connection.shouldCloseConnection.get()) {
-        throw new ConnectionClosingException("Call id=" + call.id +
-            " on server " + addr + " aborted: connection is closing");
-      }
-      try {
-        synchronized (call) {
-          if (call.done){
-            break;
-          }
-          call.wait(Math.min(call.remainingTime(), 1000) + 1);
-        }
-      } catch (InterruptedException e) {
-        call.setException(new InterruptedIOException());
-        if (cts != null) {
-          connection.callSender.remove(cts);
-        }
-        throw e;
-      }
-    }
-
-    if (call.error != null) {
-      if (call.error instanceof RemoteException) {
-        call.error.fillInStackTrace();
-        throw call.error;
-      }
-      // local exception
-      throw IPCUtil.wrapException(addr, call.error);
-    }
-
-    return call;
-  }
-
-  @Override
-  public RpcChannel createProtobufRpcChannel(ServerName sn, User user, int rpcTimeout) {
-    throw new UnsupportedOperationException();
-  }
-
-  /**
-   * Interrupt the connections to the given ip:port server. This should be called if the server
-   *  is known as actually dead. This will not prevent current operation to be retried, and,
-   *  depending on their own behavior, they may retry on the same server. This can be a feature,
-   *  for example at startup. In any case, they're likely to get connection refused (if the
-   *  process died) or no route to host: i.e. their next retries should be faster and with a
-   *  safe exception.
-   */
-  @Override
-  public void cancelConnections(ServerName sn) {
-    synchronized (connections) {
-      for (Connection connection : connections.values()) {
-        if (connection.isAlive() &&
-            connection.getRemoteAddress().getPort() == sn.getPort() &&
-            connection.getRemoteAddress().getHostName().equals(sn.getHostname())) {
-          LOG.info("The server on " + sn.toString() +
-              " is dead - stopping the connection " + connection.remoteId);
-          connection.interrupt(); // We're interrupting a Reader. It means we want it to finish.
-                                  // This will close the connection as well.
-        }
-      }
-    }
-  }
-
-  /**
-   * Get a connection from the pool, or create a new one and add it to the
-   * pool. Connections to a given host/port are reused.
-   */
-  protected Connection getConnection(User ticket, Call call, InetSocketAddress addr)
-  throws IOException {
-    if (!running.get()){
-      throw new StoppedRpcClientException();
-    }
-    Connection connection;
-    ConnectionId remoteId =
-      new ConnectionId(ticket, call.md.getService().getName(), addr);
-    synchronized (connections) {
-      connection = connections.get(remoteId);
-      if (connection == null) {
-        connection = createConnection(remoteId, this.codec, this.compressor);
-        connections.put(remoteId, connection);
-      }
-    }
-
-    return connection;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcConnection.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcConnection.java
new file mode 100644
index 0000000..8118b20
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcConnection.java
@@ -0,0 +1,255 @@
+/**
+ * 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.ipc;
+
+import io.netty.util.HashedWheelTimer;
+import io.netty.util.Timeout;
+import io.netty.util.TimerTask;
+
+import java.io.IOException;
+import java.net.UnknownHostException;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.codec.Codec;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos;
+import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ConnectionHeader;
+import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation;
+import org.apache.hadoop.hbase.security.AuthMethod;
+import org.apache.hadoop.hbase.security.SecurityInfo;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.hadoop.security.token.TokenSelector;
+
+/**
+ * Base class for ipc connection.
+ */
+@InterfaceAudience.Private
+abstract class RpcConnection {
+
+  private static final Log LOG = LogFactory.getLog(RpcConnection.class);
+
+  protected final ConnectionId remoteId;
+
+  protected final AuthMethod authMethod;
+
+  protected final boolean useSasl;
+
+  protected final Token<? extends TokenIdentifier> token;
+
+  protected final String serverPrincipal; // server's krb5 principal name
+
+  protected final int reloginMaxBackoff; // max pause before relogin on sasl failure
+
+  protected final Codec codec;
+
+  protected final CompressionCodec compressor;
+
+  protected final HashedWheelTimer timeoutTimer;
+
+  // the last time we were picked up from connection pool.
+  protected long lastTouched;
+
+  protected RpcConnection(Configuration conf, HashedWheelTimer timeoutTimer, ConnectionId remoteId,
+      String clusterId, boolean isSecurityEnabled, Codec codec, CompressionCodec compressor)
+      throws IOException {
+    if (remoteId.getAddress().isUnresolved()) {
+      throw new UnknownHostException("unknown host: " + remoteId.getAddress().getHostName());
+    }
+    this.timeoutTimer = timeoutTimer;
+    this.codec = codec;
+    this.compressor = compressor;
+
+    UserGroupInformation ticket = remoteId.getTicket().getUGI();
+    SecurityInfo securityInfo = SecurityInfo.getInfo(remoteId.getServiceName());
+    this.useSasl = isSecurityEnabled;
+    Token<? extends TokenIdentifier> token = null;
+    String serverPrincipal = null;
+    if (useSasl && securityInfo != null) {
+      AuthenticationProtos.TokenIdentifier.Kind tokenKind = securityInfo.getTokenKind();
+      if (tokenKind != null) {
+        TokenSelector<? extends TokenIdentifier> tokenSelector = AbstractRpcClient.TOKEN_HANDLERS
+            .get(tokenKind);
+        if (tokenSelector != null) {
+          token = tokenSelector.selectToken(new Text(clusterId), ticket.getTokens());
+        } else if (LOG.isDebugEnabled()) {
+          LOG.debug("No token selector found for type " + tokenKind);
+        }
+      }
+      String serverKey = securityInfo.getServerPrincipal();
+      if (serverKey == null) {
+        throw new IOException("Can't obtain server Kerberos config key from SecurityInfo");
+      }
+      serverPrincipal = SecurityUtil.getServerPrincipal(conf.get(serverKey),
+        remoteId.address.getAddress().getCanonicalHostName().toLowerCase());
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("RPC Server Kerberos principal name for service=" + remoteId.getServiceName()
+            + " is " + serverPrincipal);
+      }
+    }
+    this.token = token;
+    this.serverPrincipal = serverPrincipal;
+    if (!useSasl) {
+      authMethod = AuthMethod.SIMPLE;
+    } else if (token != null) {
+      authMethod = AuthMethod.DIGEST;
+    } else {
+      authMethod = AuthMethod.KERBEROS;
+    }
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Use " + authMethod + " authentication for service " + remoteId.serviceName
+          + ", sasl=" + useSasl);
+    }
+    reloginMaxBackoff = conf.getInt("hbase.security.relogin.maxbackoff", 5000);
+    this.remoteId = remoteId;
+  }
+
+  private UserInformation getUserInfo(UserGroupInformation ugi) {
+    if (ugi == null || authMethod == AuthMethod.DIGEST) {
+      // Don't send user for token auth
+      return null;
+    }
+    UserInformation.Builder userInfoPB = UserInformation.newBuilder();
+    if (authMethod == AuthMethod.KERBEROS) {
+      // Send effective user for Kerberos auth
+      userInfoPB.setEffectiveUser(ugi.getUserName());
+    } else if (authMethod == AuthMethod.SIMPLE) {
+      // Send both effective user and real user for simple auth
+      userInfoPB.setEffectiveUser(ugi.getUserName());
+      if (ugi.getRealUser() != null) {
+        userInfoPB.setRealUser(ugi.getRealUser().getUserName());
+      }
+    }
+    return userInfoPB.build();
+  }
+
+  protected UserGroupInformation getUGI() {
+    UserGroupInformation ticket = remoteId.getTicket().getUGI();
+    if (authMethod == AuthMethod.KERBEROS) {
+      if (ticket != null && ticket.getRealUser() != null) {
+        ticket = ticket.getRealUser();
+      }
+    }
+    return ticket;
+  }
+
+  protected boolean shouldAuthenticateOverKrb() throws IOException {
+    UserGroupInformation loginUser = UserGroupInformation.getLoginUser();
+    UserGroupInformation currentUser = UserGroupInformation.getCurrentUser();
+    UserGroupInformation realUser = currentUser.getRealUser();
+    return authMethod == AuthMethod.KERBEROS && loginUser != null &&
+    // Make sure user logged in using Kerberos either keytab or TGT
+        loginUser.hasKerberosCredentials() &&
+        // relogin only in case it is the login user (e.g. JT)
+        // or superuser (like oozie).
+        (loginUser.equals(currentUser) || loginUser.equals(realUser));
+  }
+
+  protected void relogin() throws IOException {
+    if (UserGroupInformation.isLoginKeytabBased()) {
+      UserGroupInformation.getLoginUser().reloginFromKeytab();
+    } else {
+      UserGroupInformation.getLoginUser().reloginFromTicketCache();
+    }
+  }
+
+  protected void scheduleTimeoutTask(final Call call) {
+    if (call.timeout > 0) {
+      call.timeoutTask = timeoutTimer.newTimeout(new TimerTask() {
+
+        @Override
+        public void run(Timeout timeout) throws Exception {
+          call.setTimeout(new CallTimeoutException("Call id=" + call.id + ", waitTime="
+              + (EnvironmentEdgeManager.currentTime() - call.getStartTime()) + ", rpcTimetout="
+              + call.timeout));
+          callTimeout(call);
+        }
+      }, call.timeout, TimeUnit.MILLISECONDS);
+    }
+  }
+
+  protected byte[] getConnectionHeaderPreamble() {
+    // Assemble the preamble up in a buffer first and then send it. Writing individual elements,
+    // they are getting sent across piecemeal according to wireshark and then server is messing
+    // up the reading on occasion (the passed in stream is not buffered yet).
+
+    // Preamble is six bytes -- 'HBas' + VERSION + AUTH_CODE
+    int rpcHeaderLen = HConstants.RPC_HEADER.length;
+    byte[] preamble = new byte[rpcHeaderLen + 2];
+    System.arraycopy(HConstants.RPC_HEADER, 0, preamble, 0, rpcHeaderLen);
+    preamble[rpcHeaderLen] = HConstants.RPC_CURRENT_VERSION;
+    synchronized (this) {
+      preamble[rpcHeaderLen + 1] = authMethod.code;
+    }
+    return preamble;
+  }
+
+  protected ConnectionHeader getConnectionHeader() {
+    ConnectionHeader.Builder builder = ConnectionHeader.newBuilder();
+    builder.setServiceName(remoteId.getServiceName());
+    UserInformation userInfoPB;
+    if ((userInfoPB = getUserInfo(remoteId.ticket.getUGI())) != null) {
+      builder.setUserInfo(userInfoPB);
+    }
+    if (this.codec != null) {
+      builder.setCellBlockCodecClass(this.codec.getClass().getCanonicalName());
+    }
+    if (this.compressor != null) {
+      builder.setCellBlockCompressorClass(this.compressor.getClass().getCanonicalName());
+    }
+    builder.setVersionInfo(ProtobufUtil.getVersionInfo());
+    return builder.build();
+  }
+
+  protected abstract void callTimeout(Call call);
+
+  public ConnectionId remoteId() {
+    return remoteId;
+  }
+
+  public long getLastTouched() {
+    return lastTouched;
+  }
+
+  public void setLastTouched(long lastTouched) {
+    this.lastTouched = lastTouched;
+  }
+
+  /**
+   * Tell the idle connection sweeper whether we could be swept.
+   */
+  public abstract boolean isActive();
+
+  /**
+   * Just close connection. Do not need to remove from connection pool.
+   */
+  public abstract void shutdown();
+
+  public abstract void sendRequest(Call call, HBaseRpcController hrc) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AbstractHBaseSaslRpcClient.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AbstractHBaseSaslRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AbstractHBaseSaslRpcClient.java
new file mode 100644
index 0000000..cc0114b
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AbstractHBaseSaslRpcClient.java
@@ -0,0 +1,197 @@
+/**
+ * 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.security;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import java.io.IOException;
+import java.util.Map;
+
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.CallbackHandler;
+import javax.security.auth.callback.NameCallback;
+import javax.security.auth.callback.PasswordCallback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.sasl.RealmCallback;
+import javax.security.sasl.RealmChoiceCallback;
+import javax.security.sasl.Sasl;
+import javax.security.sasl.SaslClient;
+import javax.security.sasl.SaslException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+
+/**
+ * A utility class that encapsulates SASL logic for RPC client. Copied from
+ * <code>org.apache.hadoop.security</code>
+ */
+@InterfaceAudience.Private
+public abstract class AbstractHBaseSaslRpcClient {
+
+  private static final Log LOG = LogFactory.getLog(AbstractHBaseSaslRpcClient.class);
+
+  private static final byte[] EMPTY_TOKEN = new byte[0];
+
+  protected final SaslClient saslClient;
+
+  protected final boolean fallbackAllowed;
+
+  protected final Map<String, String> saslProps;
+
+  /**
+   * Create a HBaseSaslRpcClient for an authentication method
+   * @param method the requested authentication method
+   * @param token token to use if needed by the authentication method
+   * @param serverPrincipal the server principal that we are trying to set the connection up to
+   * @param fallbackAllowed does the client allow fallback to simple authentication
+   * @throws IOException
+   */
+  protected AbstractHBaseSaslRpcClient(AuthMethod method, Token<? extends TokenIdentifier> token,
+      String serverPrincipal, boolean fallbackAllowed) throws IOException {
+    this(method, token, serverPrincipal, fallbackAllowed, "authentication");
+  }
+
+  /**
+   * Create a HBaseSaslRpcClient for an authentication method
+   * @param method the requested authentication method
+   * @param token token to use if needed by the authentication method
+   * @param serverPrincipal the server principal that we are trying to set the connection up to
+   * @param fallbackAllowed does the client allow fallback to simple authentication
+   * @param rpcProtection the protection level ("authentication", "integrity" or "privacy")
+   * @throws IOException
+   */
+  protected AbstractHBaseSaslRpcClient(AuthMethod method, Token<? extends TokenIdentifier> token,
+      String serverPrincipal, boolean fallbackAllowed, String rpcProtection) throws IOException {
+    this.fallbackAllowed = fallbackAllowed;
+    saslProps = SaslUtil.initSaslProperties(rpcProtection);
+    switch (method) {
+      case DIGEST:
+        if (LOG.isDebugEnabled()) LOG.debug("Creating SASL " + AuthMethod.DIGEST.getMechanismName()
+            + " client to authenticate to service at " + token.getService());
+        saslClient = createDigestSaslClient(new String[] { AuthMethod.DIGEST.getMechanismName() },
+          SaslUtil.SASL_DEFAULT_REALM, new SaslClientCallbackHandler(token));
+        break;
+      case KERBEROS:
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Creating SASL " + AuthMethod.KERBEROS.getMechanismName()
+              + " client. Server's Kerberos principal name is " + serverPrincipal);
+        }
+        if (serverPrincipal == null || serverPrincipal.length() == 0) {
+          throw new IOException("Failed to specify server's Kerberos principal name");
+        }
+        String[] names = SaslUtil.splitKerberosName(serverPrincipal);
+        if (names.length != 3) {
+          throw new IOException(
+              "Kerberos principal does not have the expected format: " + serverPrincipal);
+        }
+        saslClient = createKerberosSaslClient(
+          new String[] { AuthMethod.KERBEROS.getMechanismName() }, names[0], names[1]);
+        break;
+      default:
+        throw new IOException("Unknown authentication method " + method);
+    }
+    if (saslClient == null) {
+      throw new IOException("Unable to find SASL client implementation");
+    }
+  }
+
+  protected SaslClient createDigestSaslClient(String[] mechanismNames, String saslDefaultRealm,
+      CallbackHandler saslClientCallbackHandler) throws IOException {
+    return Sasl.createSaslClient(mechanismNames, null, null, saslDefaultRealm, saslProps,
+      saslClientCallbackHandler);
+  }
+
+  protected SaslClient createKerberosSaslClient(String[] mechanismNames, String userFirstPart,
+      String userSecondPart) throws IOException {
+    return Sasl.createSaslClient(mechanismNames, null, userFirstPart, userSecondPart, saslProps,
+      null);
+  }
+
+  public byte[] getInitialResponse() throws SaslException {
+    if (saslClient.hasInitialResponse()) {
+      return saslClient.evaluateChallenge(EMPTY_TOKEN);
+    } else {
+      return EMPTY_TOKEN;
+    }
+  }
+
+  public boolean isComplete() {
+    return saslClient.isComplete();
+  }
+
+  public byte[] evaluateChallenge(byte[] challenge) throws SaslException {
+    return saslClient.evaluateChallenge(challenge);
+  }
+
+  /** Release resources used by wrapped saslClient */
+  public void dispose() {
+    SaslUtil.safeDispose(saslClient);
+  }
+
+  @VisibleForTesting
+  static class SaslClientCallbackHandler implements CallbackHandler {
+    private final String userName;
+    private final char[] userPassword;
+
+    public SaslClientCallbackHandler(Token<? extends TokenIdentifier> token) {
+      this.userName = SaslUtil.encodeIdentifier(token.getIdentifier());
+      this.userPassword = SaslUtil.encodePassword(token.getPassword());
+    }
+
+    @Override
+    public void handle(Callback[] callbacks) throws UnsupportedCallbackException {
+      NameCallback nc = null;
+      PasswordCallback pc = null;
+      RealmCallback rc = null;
+      for (Callback callback : callbacks) {
+        if (callback instanceof RealmChoiceCallback) {
+          continue;
+        } else if (callback instanceof NameCallback) {
+          nc = (NameCallback) callback;
+        } else if (callback instanceof PasswordCallback) {
+          pc = (PasswordCallback) callback;
+        } else if (callback instanceof RealmCallback) {
+          rc = (RealmCallback) callback;
+        } else {
+          throw new UnsupportedCallbackException(callback, "Unrecognized SASL client callback");
+        }
+      }
+      if (nc != null) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("SASL client callback: setting username: " + userName);
+        }
+        nc.setName(userName);
+      }
+      if (pc != null) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("SASL client callback: setting userPassword");
+        }
+        pc.setPassword(userPassword);
+      }
+      if (rc != null) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("SASL client callback: setting realm: " + rc.getDefaultText());
+        }
+        rc.setText(rc.getDefaultText());
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AsyncHBaseSaslRpcClient.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AsyncHBaseSaslRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AsyncHBaseSaslRpcClient.java
new file mode 100644
index 0000000..df703dc
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AsyncHBaseSaslRpcClient.java
@@ -0,0 +1,58 @@
+/**
+ * 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.security;
+
+import io.netty.channel.ChannelPipeline;
+import io.netty.handler.codec.LengthFieldBasedFrameDecoder;
+
+import java.io.IOException;
+
+import javax.security.sasl.Sasl;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+
+/**
+ * Implement SASL logic for async rpc client.
+ */
+@InterfaceAudience.Private
+public class AsyncHBaseSaslRpcClient extends AbstractHBaseSaslRpcClient {
+  private static final Log LOG = LogFactory.getLog(AsyncHBaseSaslRpcClient.class);
+
+  public AsyncHBaseSaslRpcClient(AuthMethod method, Token<? extends TokenIdentifier> token,
+      String serverPrincipal, boolean fallbackAllowed, String rpcProtection) throws IOException {
+    super(method, token, serverPrincipal, fallbackAllowed, rpcProtection);
+  }
+
+  public void setupSaslHandler(ChannelPipeline p) {
+    String qop = (String) saslClient.getNegotiatedProperty(Sasl.QOP);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("SASL client context established. Negotiated QoP: " + qop);
+    }
+    if (qop == null || "auth".equalsIgnoreCase(qop)) {
+      return;
+    }
+    // add wrap and unwrap handlers to pipeline.
+    p.addFirst(new SaslWrapHandler(saslClient),
+      new LengthFieldBasedFrameDecoder(Integer.MAX_VALUE, 0, 4, 0, 4),
+      new SaslUnwrapHandler(saslClient));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AsyncHBaseSaslRpcClientHandler.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AsyncHBaseSaslRpcClientHandler.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AsyncHBaseSaslRpcClientHandler.java
new file mode 100644
index 0000000..bccfa30
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AsyncHBaseSaslRpcClientHandler.java
@@ -0,0 +1,135 @@
+/**
+ * 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.security;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.SimpleChannelInboundHandler;
+import io.netty.util.concurrent.Promise;
+
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.ipc.FallbackDisallowedException;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+
+/**
+ * Implement SASL logic for async rpc client.
+ */
+@InterfaceAudience.Private
+public class AsyncHBaseSaslRpcClientHandler extends SimpleChannelInboundHandler<ByteBuf> {
+
+  private static final Log LOG = LogFactory.getLog(AsyncHBaseSaslRpcClientHandler.class);
+
+  private final Promise<Boolean> saslPromise;
+
+  private final UserGroupInformation ugi;
+
+  private final AsyncHBaseSaslRpcClient saslRpcClient;
+
+  /**
+   * @param saslPromise {@code true} if success, {@code false} if server tells us to fallback to
+   *          simple.
+   */
+  public AsyncHBaseSaslRpcClientHandler(Promise<Boolean> saslPromise, UserGroupInformation ugi,
+      AuthMethod method, Token<? extends TokenIdentifier> token, String serverPrincipal,
+      boolean fallbackAllowed, String rpcProtection) throws IOException {
+    this.saslPromise = saslPromise;
+    this.ugi = ugi;
+    this.saslRpcClient = new AsyncHBaseSaslRpcClient(method, token, serverPrincipal,
+        fallbackAllowed, rpcProtection);
+  }
+
+  private void writeResponse(ChannelHandlerContext ctx, byte[] response) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Will send token of size " + response.length + " from initSASLContext.");
+    }
+    ctx.writeAndFlush(
+      ctx.alloc().buffer(4 + response.length).writeInt(response.length).writeBytes(response));
+  }
+
+  private void tryComplete(ChannelHandlerContext ctx) {
+    if (!saslRpcClient.isComplete()) {
+      return;
+    }
+    saslRpcClient.setupSaslHandler(ctx.pipeline());
+    saslPromise.setSuccess(true);
+  }
+
+  @Override
+  public void handlerAdded(ChannelHandlerContext ctx) {
+    try {
+      byte[] initialResponse = ugi.doAs(new PrivilegedExceptionAction<byte[]>() {
+
+        @Override
+        public byte[] run() throws Exception {
+          return saslRpcClient.getInitialResponse();
+        }
+      });
+      if (initialResponse != null) {
+        writeResponse(ctx, initialResponse);
+      }
+      tryComplete(ctx);
+    } catch (Exception e) {
+      // the exception thrown by handlerAdded will not be passed to the exceptionCaught below
+      // because netty will remove a handler if handlerAdded throws an exception.
+      exceptionCaught(ctx, e);
+    }
+  }
+
+  @Override
+  protected void channelRead0(ChannelHandlerContext ctx, ByteBuf msg) throws Exception {
+    int len = msg.readInt();
+    if (len == SaslUtil.SWITCH_TO_SIMPLE_AUTH) {
+      saslRpcClient.dispose();
+      if (saslRpcClient.fallbackAllowed) {
+        saslPromise.setSuccess(false);
+      } else {
+        saslPromise.setFailure(new FallbackDisallowedException());
+      }
+      return;
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Will read input token of size " + len + " for processing by initSASLContext");
+    }
+    final byte[] challenge = new byte[len];
+    msg.readBytes(challenge);
+    byte[] response = ugi.doAs(new PrivilegedExceptionAction<byte[]>() {
+
+      @Override
+      public byte[] run() throws Exception {
+        return saslRpcClient.evaluateChallenge(challenge);
+      }
+    });
+    if (response != null) {
+      writeResponse(ctx, response);
+    }
+    tryComplete(ctx);
+  }
+
+  @Override
+  public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) {
+    saslRpcClient.dispose();
+    saslPromise.setFailure(cause);
+  }
+}


[2/6] hbase git commit: HBASE-16445 Refactor and reimplement RpcClient

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-client/src/main/java/org/apache/hadoop/hbase/security/HBaseSaslRpcClient.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/HBaseSaslRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/HBaseSaslRpcClient.java
index ce32ed9..3f43f7f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/HBaseSaslRpcClient.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/HBaseSaslRpcClient.java
@@ -25,17 +25,8 @@ import java.io.DataOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
-import java.util.Map;
 
-import javax.security.auth.callback.Callback;
-import javax.security.auth.callback.CallbackHandler;
-import javax.security.auth.callback.NameCallback;
-import javax.security.auth.callback.PasswordCallback;
-import javax.security.auth.callback.UnsupportedCallbackException;
-import javax.security.sasl.RealmCallback;
-import javax.security.sasl.RealmChoiceCallback;
 import javax.security.sasl.Sasl;
-import javax.security.sasl.SaslClient;
 import javax.security.sasl.SaslException;
 
 import org.apache.commons.logging.Log;
@@ -48,105 +39,23 @@ import org.apache.hadoop.security.SaslOutputStream;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
 
-import com.google.common.annotations.VisibleForTesting;
-
 /**
- * A utility class that encapsulates SASL logic for RPC client.
- * Copied from <code>org.apache.hadoop.security</code>
+ * A utility class that encapsulates SASL logic for RPC client. Copied from
+ * <code>org.apache.hadoop.security</code>
  */
 @InterfaceAudience.Private
-public class HBaseSaslRpcClient {
-  private static final Log LOG = LogFactory.getLog(HBaseSaslRpcClient.class);
+public class HBaseSaslRpcClient extends AbstractHBaseSaslRpcClient {
 
-  private final SaslClient saslClient;
-  private final boolean fallbackAllowed;
-  protected final Map<String, String> saslProps;
-  /**
-   * Create a HBaseSaslRpcClient for an authentication method
-   *
-   * @param method
-   *          the requested authentication method
-   * @param token
-   *          token to use if needed by the authentication method
-   * @param serverPrincipal
-   *          the server principal that we are trying to set the connection up to
-   * @param fallbackAllowed
-   *          does the client allow fallback to simple authentication
-   * @throws IOException
-   */
-  public HBaseSaslRpcClient(AuthMethod method,
-      Token<? extends TokenIdentifier> token, String serverPrincipal, boolean fallbackAllowed)
-      throws IOException {
-    this(method, token, serverPrincipal, fallbackAllowed, "authentication");
-  }
-  /**
-   * Create a HBaseSaslRpcClient for an authentication method
-   *
-   * @param method
-   *          the requested authentication method
-   * @param token
-   *          token to use if needed by the authentication method
-   * @param serverPrincipal
-   *          the server principal that we are trying to set the connection up to
-   * @param fallbackAllowed
-   *          does the client allow fallback to simple authentication
-   * @param rpcProtection
-   *          the protection level ("authentication", "integrity" or "privacy")
-   * @throws IOException
-   */
-  public HBaseSaslRpcClient(AuthMethod method,
-      Token<? extends TokenIdentifier> token, String serverPrincipal, boolean fallbackAllowed,
-      String rpcProtection) throws IOException {
-    this.fallbackAllowed = fallbackAllowed;
-    saslProps = SaslUtil.initSaslProperties(rpcProtection);
-    switch (method) {
-    case DIGEST:
-      if (LOG.isDebugEnabled())
-        LOG.debug("Creating SASL " + AuthMethod.DIGEST.getMechanismName()
-            + " client to authenticate to service at " + token.getService());
-      saslClient = createDigestSaslClient(
-          new String[] { AuthMethod.DIGEST.getMechanismName() },
-          SaslUtil.SASL_DEFAULT_REALM, new SaslClientCallbackHandler(token));
-      break;
-    case KERBEROS:
-      if (LOG.isDebugEnabled()) {
-        LOG
-            .debug("Creating SASL " + AuthMethod.KERBEROS.getMechanismName()
-                + " client. Server's Kerberos principal name is "
-                + serverPrincipal);
-      }
-      if (serverPrincipal == null || serverPrincipal.length() == 0) {
-        throw new IOException(
-            "Failed to specify server's Kerberos principal name");
-      }
-      String[] names = SaslUtil.splitKerberosName(serverPrincipal);
-      if (names.length != 3) {
-        throw new IOException(
-          "Kerberos principal does not have the expected format: "
-                + serverPrincipal);
-      }
-      saslClient = createKerberosSaslClient(
-          new String[] { AuthMethod.KERBEROS.getMechanismName() },
-          names[0], names[1]);
-      break;
-    default:
-      throw new IOException("Unknown authentication method " + method);
-    }
-    if (saslClient == null)
-      throw new IOException("Unable to find SASL client implementation");
-  }
+  private static final Log LOG = LogFactory.getLog(HBaseSaslRpcClient.class);
 
-  protected SaslClient createDigestSaslClient(String[] mechanismNames,
-      String saslDefaultRealm, CallbackHandler saslClientCallbackHandler)
-      throws IOException {
-    return Sasl.createSaslClient(mechanismNames, null, null, saslDefaultRealm,
-        saslProps, saslClientCallbackHandler);
+  public HBaseSaslRpcClient(AuthMethod method, Token<? extends TokenIdentifier> token,
+      String serverPrincipal, boolean fallbackAllowed) throws IOException {
+    super(method, token, serverPrincipal, fallbackAllowed);
   }
 
-  protected SaslClient createKerberosSaslClient(String[] mechanismNames,
-      String userFirstPart, String userSecondPart) throws IOException {
-    return Sasl.createSaslClient(mechanismNames, null, userFirstPart,
-        userSecondPart, saslProps, null);
+  public HBaseSaslRpcClient(AuthMethod method, Token<? extends TokenIdentifier> token,
+      String serverPrincipal, boolean fallbackAllowed, String rpcProtection) throws IOException {
+    super(method, token, serverPrincipal, fallbackAllowed, rpcProtection);
   }
 
   private static void readStatus(DataInputStream inStream) throws IOException {
@@ -158,72 +67,65 @@ public class HBaseSaslRpcClient {
   }
 
   /**
-   * Do client side SASL authentication with server via the given InputStream
-   * and OutputStream
-   *
-   * @param inS
-   *          InputStream to use
-   * @param outS
-   *          OutputStream to use
-   * @return true if connection is set up, or false if needs to switch
-   *             to simple Auth.
+   * Do client side SASL authentication with server via the given InputStream and OutputStream
+   * @param inS InputStream to use
+   * @param outS OutputStream to use
+   * @return true if connection is set up, or false if needs to switch to simple Auth.
    * @throws IOException
    */
-  public boolean saslConnect(InputStream inS, OutputStream outS)
-      throws IOException {
+  public boolean saslConnect(InputStream inS, OutputStream outS) throws IOException {
     DataInputStream inStream = new DataInputStream(new BufferedInputStream(inS));
-    DataOutputStream outStream = new DataOutputStream(new BufferedOutputStream(
-        outS));
+    DataOutputStream outStream = new DataOutputStream(new BufferedOutputStream(outS));
 
     try {
-      byte[] saslToken = new byte[0];
-      if (saslClient.hasInitialResponse())
-        saslToken = saslClient.evaluateChallenge(saslToken);
+      byte[] saslToken = getInitialResponse();
       if (saslToken != null) {
         outStream.writeInt(saslToken.length);
         outStream.write(saslToken, 0, saslToken.length);
         outStream.flush();
-        if (LOG.isDebugEnabled())
-          LOG.debug("Have sent token of size " + saslToken.length
-              + " from initSASLContext.");
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Have sent token of size " + saslToken.length + " from initSASLContext.");
+        }
       }
-      if (!saslClient.isComplete()) {
+      if (!isComplete()) {
         readStatus(inStream);
         int len = inStream.readInt();
         if (len == SaslUtil.SWITCH_TO_SIMPLE_AUTH) {
           if (!fallbackAllowed) {
-            throw new IOException("Server asks us to fall back to SIMPLE auth, " +
-                "but this client is configured to only allow secure connections.");
+            throw new IOException("Server asks us to fall back to SIMPLE auth, "
+                + "but this client is configured to only allow secure connections.");
           }
           if (LOG.isDebugEnabled()) {
             LOG.debug("Server asks us to fall back to simple auth.");
           }
-          saslClient.dispose();
+          dispose();
           return false;
         }
         saslToken = new byte[len];
-        if (LOG.isDebugEnabled())
+        if (LOG.isDebugEnabled()) {
           LOG.debug("Will read input token of size " + saslToken.length
               + " for processing by initSASLContext");
+        }
         inStream.readFully(saslToken);
       }
 
-      while (!saslClient.isComplete()) {
-        saslToken = saslClient.evaluateChallenge(saslToken);
+      while (!isComplete()) {
+        saslToken = evaluateChallenge(saslToken);
         if (saslToken != null) {
-          if (LOG.isDebugEnabled())
-            LOG.debug("Will send token of size " + saslToken.length
-                + " from initSASLContext.");
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Will send token of size " + saslToken.length + " from initSASLContext.");
+          }
           outStream.writeInt(saslToken.length);
           outStream.write(saslToken, 0, saslToken.length);
           outStream.flush();
         }
-        if (!saslClient.isComplete()) {
+        if (!isComplete()) {
           readStatus(inStream);
           saslToken = new byte[inStream.readInt()];
-          if (LOG.isDebugEnabled())
+          if (LOG.isDebugEnabled()) {
             LOG.debug("Will read input token of size " + saslToken.length
                 + " for processing by initSASLContext");
+          }
           inStream.readFully(saslToken);
         }
       }
@@ -243,11 +145,8 @@ public class HBaseSaslRpcClient {
   }
 
   /**
-   * Get a SASL wrapped InputStream. Can be called only after saslConnect() has
-   * been called.
-   *
-   * @param in
-   *          the InputStream to wrap
+   * Get a SASL wrapped InputStream. Can be called only after saslConnect() has been called.
+   * @param in the InputStream to wrap
    * @return a SASL wrapped InputStream
    * @throws IOException
    */
@@ -259,11 +158,8 @@ public class HBaseSaslRpcClient {
   }
 
   /**
-   * Get a SASL wrapped OutputStream. Can be called only after saslConnect() has
-   * been called.
-   *
-   * @param out
-   *          the OutputStream to wrap
+   * Get a SASL wrapped OutputStream. Can be called only after saslConnect() has been called.
+   * @param out the OutputStream to wrap
    * @return a SASL wrapped OutputStream
    * @throws IOException
    */
@@ -273,58 +169,4 @@ public class HBaseSaslRpcClient {
     }
     return new SaslOutputStream(out, saslClient);
   }
-
-  /** Release resources used by wrapped saslClient */
-  public void dispose() throws SaslException {
-    saslClient.dispose();
-  }
-
-  @VisibleForTesting
-  static class SaslClientCallbackHandler implements CallbackHandler {
-    private final String userName;
-    private final char[] userPassword;
-
-    public SaslClientCallbackHandler(Token<? extends TokenIdentifier> token) {
-      this.userName = SaslUtil.encodeIdentifier(token.getIdentifier());
-      this.userPassword = SaslUtil.encodePassword(token.getPassword());
-    }
-
-    @Override
-    public void handle(Callback[] callbacks)
-        throws UnsupportedCallbackException {
-      NameCallback nc = null;
-      PasswordCallback pc = null;
-      RealmCallback rc = null;
-      for (Callback callback : callbacks) {
-        if (callback instanceof RealmChoiceCallback) {
-          continue;
-        } else if (callback instanceof NameCallback) {
-          nc = (NameCallback) callback;
-        } else if (callback instanceof PasswordCallback) {
-          pc = (PasswordCallback) callback;
-        } else if (callback instanceof RealmCallback) {
-          rc = (RealmCallback) callback;
-        } else {
-          throw new UnsupportedCallbackException(callback,
-              "Unrecognized SASL client callback");
-        }
-      }
-      if (nc != null) {
-        if (LOG.isDebugEnabled())
-          LOG.debug("SASL client callback: setting username: " + userName);
-        nc.setName(userName);
-      }
-      if (pc != null) {
-        if (LOG.isDebugEnabled())
-          LOG.debug("SASL client callback: setting userPassword");
-        pc.setPassword(userPassword);
-      }
-      if (rc != null) {
-        if (LOG.isDebugEnabled())
-          LOG.debug("SASL client callback: setting realm: "
-              + rc.getDefaultText());
-        rc.setText(rc.getDefaultText());
-      }
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslChallengeDecoder.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslChallengeDecoder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslChallengeDecoder.java
new file mode 100644
index 0000000..d818097
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslChallengeDecoder.java
@@ -0,0 +1,112 @@
+/**
+ * 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.security;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.handler.codec.ByteToMessageDecoder;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.ipc.RemoteException;
+
+/**
+ * Decode the sasl challenge sent by RpcServer.
+ */
+@InterfaceAudience.Private
+public class SaslChallengeDecoder extends ByteToMessageDecoder {
+
+  private static final int MAX_CHALLENGE_SIZE = 1024 * 1024; // 1M
+
+  private ByteBuf tryDecodeChallenge(ByteBuf in, int offset, int readableBytes) throws IOException {
+    if (readableBytes < 4) {
+      return null;
+    }
+    int len = in.getInt(offset);
+    if (len <= 0) {
+      // fall back to simple
+      in.readerIndex(offset + 4);
+      return in.retainedSlice(offset, 4);
+    }
+    if (len > MAX_CHALLENGE_SIZE) {
+      throw new IOException(
+          "Sasl challenge too large(" + len + "), max allowed is " + MAX_CHALLENGE_SIZE);
+    }
+    int totalLen = 4 + len;
+    if (readableBytes < totalLen) {
+      return null;
+    }
+    in.readerIndex(offset + totalLen);
+    return in.retainedSlice(offset, totalLen);
+  }
+
+  // will throw a RemoteException out if data is enough, so do not need to return anything.
+  private void tryDecodeError(ByteBuf in, int offset, int readableBytes) throws IOException {
+    if (readableBytes < 4) {
+      return;
+    }
+    int classLen = in.getInt(offset);
+    if (classLen <= 0) {
+      throw new IOException("Invalid exception class name length " + classLen);
+    }
+    if (classLen > MAX_CHALLENGE_SIZE) {
+      throw new IOException("Exception class name length too large(" + classLen
+          + "), max allowed is " + MAX_CHALLENGE_SIZE);
+    }
+    if (readableBytes < 4 + classLen + 4) {
+      return;
+    }
+    int msgLen = in.getInt(offset + 4 + classLen);
+    if (msgLen <= 0) {
+      throw new IOException("Invalid exception message length " + msgLen);
+    }
+    if (msgLen > MAX_CHALLENGE_SIZE) {
+      throw new IOException("Exception message length too large(" + msgLen + "), max allowed is "
+          + MAX_CHALLENGE_SIZE);
+    }
+    int totalLen = classLen + msgLen + 8;
+    if (readableBytes < totalLen) {
+      return;
+    }
+    String className = in.toString(offset + 4, classLen, HConstants.UTF8_CHARSET);
+    String msg = in.toString(offset + classLen + 8, msgLen, HConstants.UTF8_CHARSET);
+    in.readerIndex(offset + totalLen);
+    throw new RemoteException(className, msg);
+  }
+
+  @Override
+  protected void decode(ChannelHandlerContext ctx, ByteBuf in, List<Object> out) throws Exception {
+    int readableBytes = in.readableBytes();
+    if (readableBytes < 4) {
+      return;
+    }
+    int offset = in.readerIndex();
+    int status = in.getInt(offset);
+    if (status == SaslStatus.SUCCESS.state) {
+      ByteBuf challenge = tryDecodeChallenge(in, offset + 4, readableBytes - 4);
+      if (challenge != null) {
+        out.add(challenge);
+      }
+    } else {
+      tryDecodeError(in, offset + 4, readableBytes - 4);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslClientHandler.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslClientHandler.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslClientHandler.java
deleted file mode 100644
index d583e20..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslClientHandler.java
+++ /dev/null
@@ -1,382 +0,0 @@
-/**
- * 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.security;
-
-import io.netty.buffer.ByteBuf;
-import io.netty.channel.Channel;
-import io.netty.channel.ChannelDuplexHandler;
-import io.netty.channel.ChannelFuture;
-import io.netty.channel.ChannelFutureListener;
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.ChannelPromise;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.security.token.TokenIdentifier;
-
-import javax.security.auth.callback.CallbackHandler;
-import javax.security.sasl.Sasl;
-import javax.security.sasl.SaslClient;
-import javax.security.sasl.SaslException;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.charset.Charset;
-import java.security.PrivilegedExceptionAction;
-import java.util.Map;
-import java.util.Random;
-
-/**
- * Handles Sasl connections
- */
-@InterfaceAudience.Private
-public class SaslClientHandler extends ChannelDuplexHandler {
-  private static final Log LOG = LogFactory.getLog(SaslClientHandler.class);
-
-  private final boolean fallbackAllowed;
-
-  private final UserGroupInformation ticket;
-
-  /**
-   * Used for client or server's token to send or receive from each other.
-   */
-  private final SaslClient saslClient;
-  private final Map<String, String> saslProps;
-  private final SaslExceptionHandler exceptionHandler;
-  private final SaslSuccessfulConnectHandler successfulConnectHandler;
-  private byte[] saslToken;
-  private byte[] connectionHeader;
-  private boolean firstRead = true;
-
-  private int retryCount = 0;
-  private Random random;
-
-  /**
-   * @param ticket                   the ugi
-   * @param method                   auth method
-   * @param token                    for Sasl
-   * @param serverPrincipal          Server's Kerberos principal name
-   * @param fallbackAllowed          True if server may also fall back to less secure connection
-   * @param rpcProtection            Quality of protection. Can be 'authentication', 'integrity' or
-   *                                 'privacy'.
-   * @throws java.io.IOException if handler could not be created
-   */
-  public SaslClientHandler(UserGroupInformation ticket, AuthMethod method,
-      Token<? extends TokenIdentifier> token, String serverPrincipal, boolean fallbackAllowed,
-      String rpcProtection, byte[] connectionHeader, SaslExceptionHandler exceptionHandler,
-      SaslSuccessfulConnectHandler successfulConnectHandler) throws IOException {
-    this.ticket = ticket;
-    this.fallbackAllowed = fallbackAllowed;
-    this.connectionHeader = connectionHeader;
-
-    this.exceptionHandler = exceptionHandler;
-    this.successfulConnectHandler = successfulConnectHandler;
-
-    saslProps = SaslUtil.initSaslProperties(rpcProtection);
-    switch (method) {
-    case DIGEST:
-      if (LOG.isDebugEnabled())
-        LOG.debug("Creating SASL " + AuthMethod.DIGEST.getMechanismName()
-            + " client to authenticate to service at " + token.getService());
-      saslClient = createDigestSaslClient(new String[] { AuthMethod.DIGEST.getMechanismName() },
-          SaslUtil.SASL_DEFAULT_REALM, new HBaseSaslRpcClient.SaslClientCallbackHandler(token));
-      break;
-    case KERBEROS:
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Creating SASL " + AuthMethod.KERBEROS.getMechanismName()
-            + " client. Server's Kerberos principal name is " + serverPrincipal);
-      }
-      if (serverPrincipal == null || serverPrincipal.isEmpty()) {
-        throw new IOException("Failed to specify server's Kerberos principal name");
-      }
-      String[] names = SaslUtil.splitKerberosName(serverPrincipal);
-      if (names.length != 3) {
-        throw new IOException(
-            "Kerberos principal does not have the expected format: " + serverPrincipal);
-      }
-      saslClient = createKerberosSaslClient(new String[] { AuthMethod.KERBEROS.getMechanismName() },
-          names[0], names[1]);
-      break;
-    default:
-      throw new IOException("Unknown authentication method " + method);
-    }
-    if (saslClient == null) {
-      throw new IOException("Unable to find SASL client implementation");
-    }
-  }
-
-  /**
-   * Create a Digest Sasl client
-   */
-  protected SaslClient createDigestSaslClient(String[] mechanismNames, String saslDefaultRealm,
-      CallbackHandler saslClientCallbackHandler) throws IOException {
-    return Sasl.createSaslClient(mechanismNames, null, null, saslDefaultRealm, saslProps,
-        saslClientCallbackHandler);
-  }
-
-  /**
-   * Create Kerberos client
-   *
-   * @param userFirstPart  first part of username
-   * @param userSecondPart second part of username
-   */
-  protected SaslClient createKerberosSaslClient(String[] mechanismNames, String userFirstPart,
-      String userSecondPart) throws IOException {
-    return Sasl
-        .createSaslClient(mechanismNames, null, userFirstPart, userSecondPart, saslProps,
-            null);
-  }
-
-  @Override
-  public void channelUnregistered(ChannelHandlerContext ctx) throws Exception {
-    saslClient.dispose();
-  }
-
-  private byte[] evaluateChallenge(final byte[] challenge) throws Exception {
-    return ticket.doAs(new PrivilegedExceptionAction<byte[]>() {
-
-      @Override
-      public byte[] run() throws Exception {
-        return saslClient.evaluateChallenge(challenge);
-      }
-    });
-  }
-
-  @Override
-  public void handlerAdded(final ChannelHandlerContext ctx) throws Exception {
-    saslToken = new byte[0];
-    if (saslClient.hasInitialResponse()) {
-      saslToken = evaluateChallenge(saslToken);
-    }
-    if (saslToken != null) {
-      writeSaslToken(ctx, saslToken);
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Have sent token of size " + saslToken.length + " from initSASLContext.");
-      }
-    }
-  }
-
-  @Override
-  public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
-    ByteBuf in = (ByteBuf) msg;
-
-    // If not complete, try to negotiate
-    if (!saslClient.isComplete()) {
-      while (!saslClient.isComplete() && in.isReadable()) {
-        readStatus(in);
-        int len = in.readInt();
-        if (firstRead) {
-          firstRead = false;
-          if (len == SaslUtil.SWITCH_TO_SIMPLE_AUTH) {
-            if (!fallbackAllowed) {
-              throw new IOException("Server asks us to fall back to SIMPLE auth, " + "but this "
-                  + "client is configured to only allow secure connections.");
-            }
-            if (LOG.isDebugEnabled()) {
-              LOG.debug("Server asks us to fall back to simple auth.");
-            }
-            saslClient.dispose();
-
-            ctx.pipeline().remove(this);
-            successfulConnectHandler.onSuccess(ctx.channel());
-            return;
-          }
-        }
-        saslToken = new byte[len];
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Will read input token of size " + saslToken.length
-              + " for processing by initSASLContext");
-        }
-        in.readBytes(saslToken);
-
-        saslToken = evaluateChallenge(saslToken);
-        if (saslToken != null) {
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("Will send token of size " + saslToken.length + " from initSASLContext.");
-          }
-          writeSaslToken(ctx, saslToken);
-        }
-      }
-      // release the memory
-      in.release();
-
-      if (saslClient.isComplete()) {
-        String qop = (String) saslClient.getNegotiatedProperty(Sasl.QOP);
-
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("SASL client context established. Negotiated QoP: " + qop);
-        }
-
-        boolean useWrap = qop != null && !"auth".equalsIgnoreCase(qop);
-
-        if (!useWrap) {
-          ctx.pipeline().remove(this);
-          successfulConnectHandler.onSuccess(ctx.channel());
-        } else {
-          byte[] wrappedCH = saslClient.wrap(connectionHeader, 0, connectionHeader.length);
-          // write connection header
-          writeSaslToken(ctx, wrappedCH);
-          successfulConnectHandler.onSaslProtectionSucess(ctx.channel());
-        }
-      }
-    }
-    // Normal wrapped reading
-    else {
-      try {
-        int length = in.readInt();
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Actual length is " + length);
-        }
-        saslToken = new byte[length];
-        in.readBytes(saslToken);
-        // release the memory
-        in.release();
-      } catch (IndexOutOfBoundsException e) {
-        return;
-      }
-      try {
-        ByteBuf b = ctx.channel().alloc().buffer(saslToken.length);
-
-        b.writeBytes(saslClient.unwrap(saslToken, 0, saslToken.length));
-        ctx.fireChannelRead(b);
-
-      } catch (SaslException se) {
-        try {
-          saslClient.dispose();
-        } catch (SaslException ignored) {
-          LOG.debug("Ignoring SASL exception", ignored);
-        }
-        throw se;
-      }
-    }
-  }
-
-  private void writeSaslToken(final ChannelHandlerContext ctx, byte[] saslToken) {
-    ByteBuf b = ctx.alloc().buffer(4 + saslToken.length);
-    b.writeInt(saslToken.length);
-    b.writeBytes(saslToken, 0, saslToken.length);
-    ctx.writeAndFlush(b).addListener(new ChannelFutureListener() {
-      @Override
-      public void operationComplete(ChannelFuture future) throws Exception {
-        if (!future.isSuccess()) {
-          exceptionCaught(ctx, future.cause());
-        }
-      }
-    });
-  }
-
-  /**
-   * Get the read status
-   */
-  private static void readStatus(ByteBuf inStream) throws RemoteException {
-    int status = inStream.readInt(); // read status
-    if (status != SaslStatus.SUCCESS.state) {
-      throw new RemoteException(inStream.toString(Charset.forName("UTF-8")),
-          inStream.toString(Charset.forName("UTF-8")));
-    }
-  }
-
-  @Override public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause)
-      throws Exception {
-    saslClient.dispose();
-
-    ctx.close();
-
-    if (this.random == null) {
-      this.random = new Random();
-    }
-    exceptionHandler.handle(this.retryCount++, this.random, cause);
-  }
-
-  @Override
-  public void write(final ChannelHandlerContext ctx, Object msg, ChannelPromise promise)
-      throws Exception {
-    // If not complete, try to negotiate
-    if (!saslClient.isComplete()) {
-      super.write(ctx, msg, promise);
-    } else {
-      ByteBuf in = (ByteBuf) msg;
-      byte[] unwrapped = new byte[in.readableBytes()];
-      in.readBytes(unwrapped);
-      // release the memory
-      in.release();
-
-      try {
-        saslToken = saslClient.wrap(unwrapped, 0, unwrapped.length);
-      } catch (SaslException se) {
-        try {
-          saslClient.dispose();
-        } catch (SaslException ignored) {
-          LOG.debug("Ignoring SASL exception", ignored);
-        }
-        promise.setFailure(se);
-      }
-      if (saslToken != null) {
-        ByteBuf out = ctx.channel().alloc().buffer(4 + saslToken.length);
-        out.writeInt(saslToken.length);
-        out.writeBytes(saslToken, 0, saslToken.length);
-
-        ctx.write(out).addListener(new ChannelFutureListener() {
-          @Override public void operationComplete(ChannelFuture future) throws Exception {
-            if (!future.isSuccess()) {
-              exceptionCaught(ctx, future.cause());
-            }
-          }
-        });
-
-        saslToken = null;
-      }
-    }
-  }
-
-  /**
-   * Handler for exceptions during Sasl connection
-   */
-  public interface SaslExceptionHandler {
-    /**
-     * Handle the exception
-     *
-     * @param retryCount current retry count
-     * @param random     to create new backoff with
-     */
-    public void handle(int retryCount, Random random, Throwable cause);
-  }
-
-  /**
-   * Handler for successful connects
-   */
-  public interface SaslSuccessfulConnectHandler {
-    /**
-     * Runs on success
-     *
-     * @param channel which is successfully authenticated
-     */
-    public void onSuccess(Channel channel);
-
-    /**
-     * Runs on success if data protection used in Sasl
-     *
-     * @param channel which is successfully authenticated
-     */
-    public void onSaslProtectionSucess(Channel channel);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslUnwrapHandler.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslUnwrapHandler.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslUnwrapHandler.java
new file mode 100644
index 0000000..c2faf91
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslUnwrapHandler.java
@@ -0,0 +1,53 @@
+/**
+ * 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.security;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.SimpleChannelInboundHandler;
+
+import javax.security.sasl.SaslClient;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * Unwrap sasl messages. Should be placed after a
+ * {@link io.netty.handler.codec.LengthFieldBasedFrameDecoder}
+ */
+@InterfaceAudience.Private
+public class SaslUnwrapHandler extends SimpleChannelInboundHandler<ByteBuf> {
+
+  private final SaslClient saslClient;
+
+  public SaslUnwrapHandler(SaslClient saslClient) {
+    this.saslClient = saslClient;
+  }
+
+  @Override
+  public void channelInactive(ChannelHandlerContext ctx) throws Exception {
+    SaslUtil.safeDispose(saslClient);
+  }
+
+  @Override
+  protected void channelRead0(ChannelHandlerContext ctx, ByteBuf msg) throws Exception {
+    byte[] bytes = new byte[msg.readableBytes()];
+    msg.readBytes(bytes);
+    ctx.fireChannelRead(Unpooled.wrappedBuffer(saslClient.unwrap(bytes, 0, bytes.length)));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslUtil.java
index b505fc0..aaa9d7a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslUtil.java
@@ -22,6 +22,8 @@ import java.util.Map;
 import java.util.TreeMap;
 
 import javax.security.sasl.Sasl;
+import javax.security.sasl.SaslClient;
+import javax.security.sasl.SaslException;
 
 import org.apache.commons.codec.binary.Base64;
 import org.apache.commons.logging.Log;
@@ -30,7 +32,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 
 @InterfaceAudience.Private
 public class SaslUtil {
-  private static final Log log = LogFactory.getLog(SaslUtil.class);
+  private static final Log LOG = LogFactory.getLog(SaslUtil.class);
   public static final String SASL_DEFAULT_REALM = "default";
   public static final int SWITCH_TO_SIMPLE_AUTH = -88;
 
@@ -51,7 +53,7 @@ public class SaslUtil {
 
     public boolean matches(String stringQop) {
       if (saslQop.equals(stringQop)) {
-        log.warn("Use authentication/integrity/privacy as value for rpc protection "
+        LOG.warn("Use authentication/integrity/privacy as value for rpc protection "
             + "configurations instead of auth/auth-int/auth-conf.");
         return true;
       }
@@ -113,4 +115,12 @@ public class SaslUtil {
     saslProps.put(Sasl.SERVER_AUTH, "true");
     return saslProps;
   }
+
+  static void safeDispose(SaslClient saslClient) {
+    try {
+      saslClient.dispose();
+    } catch (SaslException e) {
+      LOG.error("Error disposing of SASL client", e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslWrapHandler.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslWrapHandler.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslWrapHandler.java
new file mode 100644
index 0000000..ddb4ae9
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslWrapHandler.java
@@ -0,0 +1,80 @@
+/**
+ * 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.security;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelOutboundHandlerAdapter;
+import io.netty.channel.ChannelPromise;
+import io.netty.channel.CoalescingBufferQueue;
+import io.netty.util.concurrent.PromiseCombiner;
+
+import javax.security.sasl.SaslClient;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * wrap sasl messages.
+ */
+@InterfaceAudience.Private
+public class SaslWrapHandler extends ChannelOutboundHandlerAdapter {
+
+  private final SaslClient saslClient;
+
+  private CoalescingBufferQueue queue;
+
+  @Override
+  public void handlerAdded(ChannelHandlerContext ctx) throws Exception {
+    queue = new CoalescingBufferQueue(ctx.channel());
+  }
+
+  @Override
+  public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise)
+      throws Exception {
+    if (msg instanceof ByteBuf) {
+      queue.add((ByteBuf) msg, promise);
+    } else {
+      ctx.write(msg, promise);
+    }
+  }
+
+  public SaslWrapHandler(SaslClient saslClient) {
+    this.saslClient = saslClient;
+  }
+
+  @Override
+  public void flush(ChannelHandlerContext ctx) throws Exception {
+    if (!queue.isEmpty()) {
+      ChannelPromise promise = ctx.newPromise();
+      int readableBytes = queue.readableBytes();
+      ByteBuf buf = queue.remove(readableBytes, promise);
+      byte[] bytes = new byte[readableBytes];
+      buf.readBytes(bytes);
+      byte[] wrapperBytes = saslClient.wrap(bytes, 0, bytes.length);
+      ChannelPromise lenPromise = ctx.newPromise();
+      ctx.write(ctx.alloc().buffer(4).writeInt(wrapperBytes.length), lenPromise);
+      ChannelPromise contentPromise = ctx.newPromise();
+      ctx.write(Unpooled.wrappedBuffer(wrapperBytes), contentPromise);
+      PromiseCombiner combiner = new PromiseCombiner();
+      combiner.addAll(lenPromise, contentPromise);
+      combiner.finish(promise);
+    }
+    ctx.flush();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestCellBlockBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestCellBlockBuilder.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestCellBlockBuilder.java
index 60ef357..ccabe66 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestCellBlockBuilder.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestCellBlockBuilder.java
@@ -27,7 +27,6 @@ import org.apache.commons.lang.time.StopWatch;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.impl.Log4JLogger;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.CellUtil;
@@ -53,11 +52,11 @@ public class TestCellBlockBuilder {
 
   private static final Log LOG = LogFactory.getLog(TestCellBlockBuilder.class);
 
-  CellBlockBuilder builder;
+  private CellBlockBuilder builder;
 
   @Before
   public void before() {
-    this.builder = new CellBlockBuilder(new Configuration());
+    this.builder = new CellBlockBuilder(HBaseConfiguration.create());
   }
 
   @Test
@@ -164,9 +163,9 @@ public class TestCellBlockBuilder {
         + count + ", size=" + size + ", + took=" + timer.getTime() + "ms");
   }
 
-  private static void timerTest(final CellBlockBuilder builder, final StopWatch timer, final int count,
-      final int size, final Codec codec, final CompressionCodec compressor, final boolean sized)
-      throws IOException {
+  private static void timerTest(final CellBlockBuilder builder, final StopWatch timer,
+      final int count, final int size, final Codec codec, final CompressionCodec compressor,
+      final boolean sized) throws IOException {
     doBuildCellBlockUndoCellBlock(builder, codec, compressor, count, size, sized);
   }
 
@@ -187,10 +186,10 @@ public class TestCellBlockBuilder {
         usage(1);
       }
     }
-    CellBlockBuilder buildr = new CellBlockBuilder(HBaseConfiguration.create());
+    CellBlockBuilder builder = new CellBlockBuilder(HBaseConfiguration.create());
     ((Log4JLogger) CellBlockBuilder.LOG).getLogger().setLevel(Level.ALL);
-    timerTests(buildr, count, size, new KeyValueCodec(), null);
-    timerTests(buildr, count, size, new KeyValueCodec(), new DefaultCodec());
-    timerTests(buildr, count, size, new KeyValueCodec(), new GzipCodec());
+    timerTests(builder, count, size, new KeyValueCodec(), null);
+    timerTests(builder, count, size, new KeyValueCodec(), new DefaultCodec());
+    timerTests(builder, count, size, new KeyValueCodec(), new GzipCodec());
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcClientDeprecatedNameMapping.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcClientDeprecatedNameMapping.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcClientDeprecatedNameMapping.java
new file mode 100644
index 0000000..7ac5c2e
--- /dev/null
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcClientDeprecatedNameMapping.java
@@ -0,0 +1,56 @@
+/**
+ * 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.ipc;
+
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.junit.Assert.assertThat;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ ClientTests.class, SmallTests.class })
+public class TestRpcClientDeprecatedNameMapping {
+
+  @Test
+  public void test() {
+    Configuration conf = HBaseConfiguration.create();
+    conf.set(RpcClientFactory.CUSTOM_RPC_CLIENT_IMPL_CONF_KEY, BlockingRpcClient.class.getName());
+    try (RpcClient client = RpcClientFactory.createClient(conf, HConstants.CLUSTER_ID_DEFAULT)) {
+      assertThat(client, instanceOf(BlockingRpcClient.class));
+    }
+    conf.set(RpcClientFactory.CUSTOM_RPC_CLIENT_IMPL_CONF_KEY,
+      "org.apache.hadoop.hbase.ipc.RpcClientImpl");
+    try (RpcClient client = RpcClientFactory.createClient(conf, HConstants.CLUSTER_ID_DEFAULT)) {
+      assertThat(client, instanceOf(BlockingRpcClient.class));
+    }
+    conf.set(RpcClientFactory.CUSTOM_RPC_CLIENT_IMPL_CONF_KEY, NettyRpcClient.class.getName());
+    try (RpcClient client = RpcClientFactory.createClient(conf, HConstants.CLUSTER_ID_DEFAULT)) {
+      assertThat(client, instanceOf(NettyRpcClient.class));
+    }
+    conf.set(RpcClientFactory.CUSTOM_RPC_CLIENT_IMPL_CONF_KEY,
+      "org.apache.hadoop.hbase.ipc.AsyncRpcClient");
+    try (RpcClient client = RpcClientFactory.createClient(conf, HConstants.CLUSTER_ID_DEFAULT)) {
+      assertThat(client, instanceOf(NettyRpcClient.class));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-client/src/test/java/org/apache/hadoop/hbase/security/TestHBaseSaslRpcClient.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/security/TestHBaseSaslRpcClient.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/security/TestHBaseSaslRpcClient.java
index 0e3aeab..12b3661 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/security/TestHBaseSaslRpcClient.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/security/TestHBaseSaslRpcClient.java
@@ -27,6 +27,8 @@ import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
+import com.google.common.base.Strings;
+
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
@@ -37,14 +39,14 @@ import javax.security.auth.callback.NameCallback;
 import javax.security.auth.callback.PasswordCallback;
 import javax.security.auth.callback.TextOutputCallback;
 import javax.security.auth.callback.UnsupportedCallbackException;
-import javax.security.sasl.Sasl;
 import javax.security.sasl.RealmCallback;
 import javax.security.sasl.RealmChoiceCallback;
+import javax.security.sasl.Sasl;
 import javax.security.sasl.SaslClient;
 
+import org.apache.hadoop.hbase.security.AbstractHBaseSaslRpcClient.SaslClientCallbackHandler;
 import org.apache.hadoop.hbase.testclassification.SecurityTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.apache.hadoop.hbase.security.HBaseSaslRpcClient.SaslClientCallbackHandler;
 import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.security.token.Token;
@@ -58,8 +60,6 @@ import org.junit.experimental.categories.Category;
 import org.junit.rules.ExpectedException;
 import org.mockito.Mockito;
 
-import com.google.common.base.Strings;
-
 @Category({SecurityTests.class, SmallTests.class})
 public class TestHBaseSaslRpcClient {
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-it/src/test/java/org/apache/hadoop/hbase/ipc/IntegrationTestRpcClient.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/ipc/IntegrationTestRpcClient.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/ipc/IntegrationTestRpcClient.java
index 28c19ad..f41efc7 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/ipc/IntegrationTestRpcClient.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/ipc/IntegrationTestRpcClient.java
@@ -47,7 +47,6 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.codec.Codec;
 import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoRequestProto;
 import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoResponseProto;
@@ -93,15 +92,13 @@ public class IntegrationTestRpcClient {
     }
   }
 
-  protected AbstractRpcClient createRpcClient(Configuration conf, boolean isSyncClient) {
-    return isSyncClient ?
-        new RpcClientImpl(conf, HConstants.CLUSTER_ID_DEFAULT) :
-          new AsyncRpcClient(conf) {
-          @Override
-          Codec getCodec() {
-            return null;
-          }
-        };
+  protected AbstractRpcClient<?> createRpcClient(Configuration conf, boolean isSyncClient) {
+    return isSyncClient ? new BlockingRpcClient(conf) : new NettyRpcClient(conf) {
+      @Override
+      Codec getCodec() {
+        return null;
+      }
+    };
   }
 
   static String BIG_PAYLOAD;
@@ -258,7 +255,7 @@ public class IntegrationTestRpcClient {
   }
 
   static class SimpleClient extends Thread {
-    AbstractRpcClient rpcClient;
+    AbstractRpcClient<?> rpcClient;
     AtomicBoolean running = new  AtomicBoolean(true);
     AtomicBoolean sending = new AtomicBoolean(false);
     AtomicReference<Throwable> exception = new AtomicReference<>(null);
@@ -267,7 +264,7 @@ public class IntegrationTestRpcClient {
     long numCalls = 0;
     Random random = new Random();
 
-    public SimpleClient(Cluster cluster, AbstractRpcClient rpcClient, String id) {
+    public SimpleClient(Cluster cluster, AbstractRpcClient<?> rpcClient, String id) {
       this.cluster = cluster;
       this.rpcClient = rpcClient;
       this.id = id;
@@ -327,7 +324,7 @@ public class IntegrationTestRpcClient {
     cluster.startServer();
     conf.setBoolean(SPECIFIC_WRITE_THREAD, true);
     for(int i = 0; i <1000; i++) {
-      AbstractRpcClient rpcClient = createRpcClient(conf, true);
+      AbstractRpcClient<?> rpcClient = createRpcClient(conf, true);
       SimpleClient client = new SimpleClient(cluster, rpcClient, "Client1");
       client.start();
       while(!client.isSending()) {
@@ -419,7 +416,7 @@ public class IntegrationTestRpcClient {
     ArrayList<SimpleClient> clients = new ArrayList<>();
 
     // all threads should share the same rpc client
-    AbstractRpcClient rpcClient = createRpcClient(conf, isSyncClient);
+    AbstractRpcClient<?> rpcClient = createRpcClient(conf, isSyncClient);
 
     for (int i = 0; i < 30; i++) {
       String clientId = "client_" + i + "_";

http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
index f611796..0df5097 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
@@ -157,7 +157,7 @@ import com.google.protobuf.TextFormat;
  * CallRunner#run executes the call.  When done, asks the included Call to put itself on new
  * queue for Responder to pull from and return result to client.
  *
- * @see RpcClientImpl
+ * @see BlockingRpcClient
  */
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 @InterfaceStability.Evolving

http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
index 663535b..f97dfb4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
@@ -863,6 +863,7 @@ public class ServerManager {
     }
     long expiration = timeout + System.currentTimeMillis();
     while (System.currentTimeMillis() < expiration) {
+      controller.reset();
       try {
         HRegionInfo rsRegion =
           ProtobufUtil.getRegionInfo(controller, rs, region.getRegionName());

http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-server/src/main/java/org/apache/hadoop/hbase/util/MultiHConnection.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/MultiHConnection.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/MultiHConnection.java
index 1688874..5934e07 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/MultiHConnection.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/MultiHConnection.java
@@ -39,7 +39,6 @@ import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Row;
 import org.apache.hadoop.hbase.client.coprocessor.Batch;
-import org.apache.hadoop.hbase.ipc.AsyncRpcClient;
 
 /**
  * Provides ability to create multiple Connection instances and allows to process a batch of

http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java
index f49c558..0349ca5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java
@@ -22,6 +22,7 @@ package org.apache.hadoop.hbase.client;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
+import java.net.InetSocketAddress;
 import java.net.SocketAddress;
 import java.net.SocketTimeoutException;
 import java.net.UnknownHostException;
@@ -36,7 +37,7 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
 import org.apache.hadoop.hbase.ipc.AbstractRpcClient;
 import org.apache.hadoop.hbase.ipc.RpcClientFactory;
-import org.apache.hadoop.hbase.ipc.RpcClientImpl;
+import org.apache.hadoop.hbase.ipc.BlockingRpcClient;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -130,7 +131,7 @@ public class TestClientTimeouts {
   /**
    * Rpc Channel implementation with RandomTimeoutBlockingRpcChannel
    */
-  public static class RandomTimeoutRpcClient extends RpcClientImpl {
+  public static class RandomTimeoutRpcClient extends BlockingRpcClient {
     public RandomTimeoutRpcClient(Configuration conf, String clusterId, SocketAddress localAddr,
         MetricsConnection metrics) {
       super(conf, clusterId, localAddr, metrics);
@@ -153,9 +154,9 @@ public class TestClientTimeouts {
     public static final double CHANCE_OF_TIMEOUT = 0.3;
     private static AtomicInteger invokations = new AtomicInteger();
 
-    RandomTimeoutBlockingRpcChannel(final RpcClientImpl rpcClient, final ServerName sn,
-        final User ticket, final int rpcTimeout) throws UnknownHostException {
-      super(rpcClient, sn, ticket, rpcTimeout);
+    RandomTimeoutBlockingRpcChannel(final BlockingRpcClient rpcClient, final ServerName sn,
+        final User ticket, final int rpcTimeout) {
+      super(rpcClient, new InetSocketAddress(sn.getHostname(), sn.getPort()), ticket, rpcTimeout);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRpcControllerFactory.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRpcControllerFactory.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRpcControllerFactory.java
index c1c9b1e..aac020d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRpcControllerFactory.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRpcControllerFactory.java
@@ -170,7 +170,7 @@ public class TestRpcControllerFactory {
     ResultScanner scan = table.getScanner(fam1);
     scan.next();
     scan.close();
-    counter = verifyCount(counter);
+    counter = verifyCount(counter + 2);
 
     Get g2 = new Get(row);
     table.get(Lists.newArrayList(g, g2));
@@ -189,7 +189,7 @@ public class TestRpcControllerFactory {
 
     // reversed, regular
     scanInfo.setSmall(false);
-    counter = doScan(table, scanInfo, counter);
+    counter = doScan(table, scanInfo, counter + 2);
 
     table.close();
     connection.close();

http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java
index 2dbe6b0..a8ea4ee 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java
@@ -19,7 +19,9 @@ package org.apache.hadoop.hbase.ipc;
 
 import static org.apache.hadoop.hbase.ipc.TestProtobufRpcServiceImpl.SERVICE;
 import static org.apache.hadoop.hbase.ipc.TestProtobufRpcServiceImpl.newBlockingStub;
+import static org.apache.hadoop.hbase.ipc.TestProtobufRpcServiceImpl.newStub;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
@@ -50,9 +52,12 @@ import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoRequestProto;
+import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoResponseProto;
 import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto;
+import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto;
 import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.PauseRequestProto;
 import org.apache.hadoop.hbase.ipc.protobuf.generated.TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface;
+import org.apache.hadoop.hbase.ipc.protobuf.generated.TestRpcServiceProtos.TestProtobufRpcProto.Interface;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.io.compress.GzipCodec;
@@ -93,7 +98,7 @@ public abstract class AbstractTestIPC {
     }
   }
 
-  protected abstract AbstractRpcClient createRpcClientNoCodec(Configuration conf);
+  protected abstract AbstractRpcClient<?> createRpcClientNoCodec(Configuration conf);
 
   /**
    * Ensure we do not HAVE TO HAVE a codec.
@@ -102,7 +107,7 @@ public abstract class AbstractTestIPC {
   public void testNoCodec() throws IOException, ServiceException {
     Configuration conf = HBaseConfiguration.create();
     TestRpcServer rpcServer = new TestRpcServer();
-    try (AbstractRpcClient client = createRpcClientNoCodec(conf)) {
+    try (AbstractRpcClient<?> client = createRpcClientNoCodec(conf)) {
       rpcServer.start();
       BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress());
       HBaseRpcController pcrc = new HBaseRpcControllerImpl();
@@ -115,7 +120,7 @@ public abstract class AbstractTestIPC {
     }
   }
 
-  protected abstract AbstractRpcClient createRpcClient(Configuration conf);
+  protected abstract AbstractRpcClient<?> createRpcClient(Configuration conf);
 
   /**
    * It is hard to verify the compression is actually happening under the wraps. Hope that if
@@ -132,7 +137,7 @@ public abstract class AbstractTestIPC {
       cells.add(CELL);
     }
     TestRpcServer rpcServer = new TestRpcServer();
-    try (AbstractRpcClient client = createRpcClient(conf)) {
+    try (AbstractRpcClient<?> client = createRpcClient(conf)) {
       rpcServer.start();
       BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress());
       HBaseRpcController pcrc = new HBaseRpcControllerImpl(CellUtil.createCellScanner(cells));
@@ -152,14 +157,14 @@ public abstract class AbstractTestIPC {
     }
   }
 
-  protected abstract AbstractRpcClient createRpcClientRTEDuringConnectionSetup(Configuration conf)
-      throws IOException;
+  protected abstract AbstractRpcClient<?> createRpcClientRTEDuringConnectionSetup(
+      Configuration conf) throws IOException;
 
   @Test
   public void testRTEDuringConnectionSetup() throws Exception {
     Configuration conf = HBaseConfiguration.create();
     TestRpcServer rpcServer = new TestRpcServer();
-    try (AbstractRpcClient client = createRpcClientRTEDuringConnectionSetup(conf)) {
+    try (AbstractRpcClient<?> client = createRpcClientRTEDuringConnectionSetup(conf)) {
       rpcServer.start();
       BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress());
       stub.ping(null, EmptyRequestProto.getDefaultInstance());
@@ -180,7 +185,7 @@ public abstract class AbstractTestIPC {
     RpcScheduler scheduler = spy(new FifoRpcScheduler(CONF, 1));
     RpcServer rpcServer = new TestRpcServer(scheduler, CONF);
     verify(scheduler).init((RpcScheduler.Context) anyObject());
-    try (AbstractRpcClient client = createRpcClient(CONF)) {
+    try (AbstractRpcClient<?> client = createRpcClient(CONF)) {
       rpcServer.start();
       verify(scheduler).start();
       BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress());
@@ -201,7 +206,7 @@ public abstract class AbstractTestIPC {
     Configuration conf = new Configuration(CONF);
     conf.setInt(RpcServer.MAX_REQUEST_SIZE, 100);
     RpcServer rpcServer = new TestRpcServer(conf);
-    try (AbstractRpcClient client = createRpcClient(conf)) {
+    try (AbstractRpcClient<?> client = createRpcClient(conf)) {
       rpcServer.start();
       BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress());
       StringBuilder message = new StringBuilder(120);
@@ -232,7 +237,7 @@ public abstract class AbstractTestIPC {
       throws IOException, ServiceException {
     TestRpcServer rpcServer = new TestRpcServer();
     InetSocketAddress localAddr = new InetSocketAddress("localhost", 0);
-    try (AbstractRpcClient client = createRpcClient(CONF)) {
+    try (AbstractRpcClient<?> client = createRpcClient(CONF)) {
       rpcServer.start();
       BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress());
       assertEquals(localAddr.getAddress().getHostAddress(),
@@ -245,12 +250,12 @@ public abstract class AbstractTestIPC {
   @Test
   public void testRemoteError() throws IOException, ServiceException {
     TestRpcServer rpcServer = new TestRpcServer();
-    try (AbstractRpcClient client = createRpcClient(CONF)) {
+    try (AbstractRpcClient<?> client = createRpcClient(CONF)) {
       rpcServer.start();
       BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress());
       stub.error(null, EmptyRequestProto.getDefaultInstance());
     } catch (ServiceException e) {
-      LOG.info("Caught expected exception: " + e.getMessage());
+      LOG.info("Caught expected exception: " + e);
       IOException ioe = ProtobufUtil.handleRemoteException(e);
       assertTrue(ioe instanceof DoNotRetryIOException);
       assertTrue(ioe.getMessage().contains("server error!"));
@@ -262,7 +267,7 @@ public abstract class AbstractTestIPC {
   @Test
   public void testTimeout() throws IOException {
     TestRpcServer rpcServer = new TestRpcServer();
-    try (AbstractRpcClient client = createRpcClient(CONF)) {
+    try (AbstractRpcClient<?> client = createRpcClient(CONF)) {
       rpcServer.start();
       BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress());
       HBaseRpcController pcrc = new HBaseRpcControllerImpl();
@@ -277,7 +282,7 @@ public abstract class AbstractTestIPC {
         } catch (ServiceException e) {
           long waitTime = (System.nanoTime() - startTime) / 1000000;
           // expected
-          LOG.info("Caught expected exception: " + e.getMessage());
+          LOG.info("Caught expected exception: " + e);
           IOException ioe = ProtobufUtil.handleRemoteException(e);
           assertTrue(ioe.getCause() instanceof CallTimeoutException);
           // confirm that we got exception before the actual pause.
@@ -327,7 +332,7 @@ public abstract class AbstractTestIPC {
   public void testConnectionCloseWithOutstandingRPCs() throws InterruptedException, IOException {
     Configuration conf = new Configuration(CONF);
     RpcServer rpcServer = new TestFailingRpcServer(conf);
-    try (AbstractRpcClient client = createRpcClient(conf)) {
+    try (AbstractRpcClient<?> client = createRpcClient(conf)) {
       rpcServer.start();
       BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress());
       EchoRequestProto param = EchoRequestProto.newBuilder().setMessage("hello").build();
@@ -339,4 +344,90 @@ public abstract class AbstractTestIPC {
       rpcServer.stop();
     }
   }
+
+  @Test
+  public void testAsyncEcho() throws IOException {
+    Configuration conf = HBaseConfiguration.create();
+    TestRpcServer rpcServer = new TestRpcServer();
+    try (AbstractRpcClient<?> client = createRpcClient(conf)) {
+      rpcServer.start();
+      Interface stub = newStub(client, rpcServer.getListenerAddress());
+      int num = 10;
+      List<HBaseRpcController> pcrcList = new ArrayList<>();
+      List<BlockingRpcCallback<EchoResponseProto>> callbackList = new ArrayList<>();
+      for (int i = 0; i < num; i++) {
+        HBaseRpcController pcrc = new HBaseRpcControllerImpl();
+        BlockingRpcCallback<EchoResponseProto> done = new BlockingRpcCallback<>();
+        stub.echo(pcrc, EchoRequestProto.newBuilder().setMessage("hello-" + i).build(), done);
+        pcrcList.add(pcrc);
+        callbackList.add(done);
+      }
+      for (int i = 0; i < num; i++) {
+        HBaseRpcController pcrc = pcrcList.get(i);
+        assertFalse(pcrc.failed());
+        assertNull(pcrc.cellScanner());
+        assertEquals("hello-" + i, callbackList.get(i).get().getMessage());
+      }
+    } finally {
+      rpcServer.stop();
+    }
+  }
+
+  @Test
+  public void testAsyncRemoteError() throws IOException {
+    AbstractRpcClient<?> client = createRpcClient(CONF);
+    TestRpcServer rpcServer = new TestRpcServer();
+    try {
+      rpcServer.start();
+      Interface stub = newStub(client, rpcServer.getListenerAddress());
+      BlockingRpcCallback<EmptyResponseProto> callback = new BlockingRpcCallback<>();
+      HBaseRpcController pcrc = new HBaseRpcControllerImpl();
+      stub.error(pcrc, EmptyRequestProto.getDefaultInstance(), callback);
+      assertNull(callback.get());
+      assertTrue(pcrc.failed());
+      LOG.info("Caught expected exception: " + pcrc.getFailed());
+      IOException ioe = ProtobufUtil.handleRemoteException(pcrc.getFailed());
+      assertTrue(ioe instanceof DoNotRetryIOException);
+      assertTrue(ioe.getMessage().contains("server error!"));
+    } finally {
+      client.close();
+      rpcServer.stop();
+    }
+  }
+
+  @Test
+  public void testAsyncTimeout() throws IOException {
+    TestRpcServer rpcServer = new TestRpcServer();
+    try (AbstractRpcClient<?> client = createRpcClient(CONF)) {
+      rpcServer.start();
+      Interface stub = newStub(client, rpcServer.getListenerAddress());
+      List<HBaseRpcController> pcrcList = new ArrayList<>();
+      List<BlockingRpcCallback<EmptyResponseProto>> callbackList = new ArrayList<>();
+      int ms = 1000;
+      int timeout = 100;
+      long startTime = System.nanoTime();
+      for (int i = 0; i < 10; i++) {
+        HBaseRpcController pcrc = new HBaseRpcControllerImpl();
+        pcrc.setCallTimeout(timeout);
+        BlockingRpcCallback<EmptyResponseProto> callback = new BlockingRpcCallback<>();
+        stub.pause(pcrc, PauseRequestProto.newBuilder().setMs(ms).build(), callback);
+        pcrcList.add(pcrc);
+        callbackList.add(callback);
+      }
+      for (BlockingRpcCallback<?> callback : callbackList) {
+        assertNull(callback.get());
+      }
+      long waitTime = (System.nanoTime() - startTime) / 1000000;
+      for (HBaseRpcController pcrc : pcrcList) {
+        assertTrue(pcrc.failed());
+        LOG.info("Caught expected exception: " + pcrc.getFailed());
+        IOException ioe = ProtobufUtil.handleRemoteException(pcrc.getFailed());
+        assertTrue(ioe.getCause() instanceof CallTimeoutException);
+      }
+      // confirm that we got exception before the actual pause.
+      assertTrue(waitTime < ms);
+    } finally {
+      rpcServer.stop();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestAsyncIPC.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestAsyncIPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestAsyncIPC.java
deleted file mode 100644
index 565f5bf..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestAsyncIPC.java
+++ /dev/null
@@ -1,113 +0,0 @@
-/**
- * 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.ipc;
-
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.ChannelInitializer;
-import io.netty.channel.ChannelOutboundHandlerAdapter;
-import io.netty.channel.ChannelPromise;
-import io.netty.channel.epoll.EpollEventLoopGroup;
-import io.netty.channel.nio.NioEventLoopGroup;
-import io.netty.channel.socket.SocketChannel;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.codec.Codec;
-import org.apache.hadoop.hbase.testclassification.RPCTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.junit.experimental.categories.Category;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
-
-@RunWith(Parameterized.class)
-@Category({ RPCTests.class, SmallTests.class })
-public class TestAsyncIPC extends AbstractTestIPC {
-
-  @Parameters
-  public static Collection<Object[]> parameters() {
-    List<Object[]> paramList = new ArrayList<>();
-    paramList.add(new Object[] { false, false });
-    paramList.add(new Object[] { false, true });
-    paramList.add(new Object[] { true, false });
-    paramList.add(new Object[] { true, true });
-    return paramList;
-  }
-
-  private final boolean useNativeTransport;
-
-  private final boolean useGlobalEventLoopGroup;
-
-  public TestAsyncIPC(boolean useNativeTransport, boolean useGlobalEventLoopGroup) {
-    this.useNativeTransport = useNativeTransport;
-    this.useGlobalEventLoopGroup = useGlobalEventLoopGroup;
-  }
-
-  private void setConf(Configuration conf) {
-    conf.setBoolean(AsyncRpcClient.USE_NATIVE_TRANSPORT, useNativeTransport);
-    conf.setBoolean(AsyncRpcClient.USE_GLOBAL_EVENT_LOOP_GROUP, useGlobalEventLoopGroup);
-    if (useGlobalEventLoopGroup && AsyncRpcClient.GLOBAL_EVENT_LOOP_GROUP != null) {
-      if (useNativeTransport
-          && !(AsyncRpcClient.GLOBAL_EVENT_LOOP_GROUP.getFirst() instanceof EpollEventLoopGroup)
-          || (!useNativeTransport && !(AsyncRpcClient.GLOBAL_EVENT_LOOP_GROUP
-              .getFirst() instanceof NioEventLoopGroup))) {
-        AsyncRpcClient.GLOBAL_EVENT_LOOP_GROUP.getFirst().shutdownGracefully();
-        AsyncRpcClient.GLOBAL_EVENT_LOOP_GROUP = null;
-      }
-    }
-  }
-
-  @Override
-  protected AsyncRpcClient createRpcClientNoCodec(Configuration conf) {
-    setConf(conf);
-    return new AsyncRpcClient(conf) {
-
-      @Override
-      Codec getCodec() {
-        return null;
-      }
-
-    };
-  }
-
-  @Override
-  protected AsyncRpcClient createRpcClient(Configuration conf) {
-    setConf(conf);
-    return new AsyncRpcClient(conf);
-  }
-
-  @Override
-  protected AsyncRpcClient createRpcClientRTEDuringConnectionSetup(Configuration conf) {
-    setConf(conf);
-    return new AsyncRpcClient(conf, new ChannelInitializer<SocketChannel>() {
-      @Override
-      protected void initChannel(SocketChannel ch) throws Exception {
-        ch.pipeline().addFirst(new ChannelOutboundHandlerAdapter() {
-          @Override
-          public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise)
-              throws Exception {
-            promise.setFailure(new RuntimeException("Injected fault"));
-          }
-        });
-      }
-    });
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestBlockingIPC.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestBlockingIPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestBlockingIPC.java
new file mode 100644
index 0000000..98efcfb
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestBlockingIPC.java
@@ -0,0 +1,58 @@
+/**
+ *
+ * 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.ipc;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.codec.Codec;
+import org.apache.hadoop.hbase.testclassification.RPCTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.experimental.categories.Category;
+
+@Category({ RPCTests.class, SmallTests.class })
+public class TestBlockingIPC extends AbstractTestIPC {
+
+  @Override
+  protected BlockingRpcClient createRpcClientNoCodec(Configuration conf) {
+    return new BlockingRpcClient(conf) {
+      @Override
+      Codec getCodec() {
+        return null;
+      }
+    };
+  }
+
+  @Override
+  protected BlockingRpcClient createRpcClient(Configuration conf) {
+    return new BlockingRpcClient(conf);
+  }
+
+  @Override
+  protected BlockingRpcClient createRpcClientRTEDuringConnectionSetup(Configuration conf)
+      throws IOException {
+    return new BlockingRpcClient(conf) {
+
+      @Override
+      boolean isTcpNoDelay() {
+        throw new RuntimeException("Injected fault");
+      }
+    };
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestGlobalEventLoopGroup.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestGlobalEventLoopGroup.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestGlobalEventLoopGroup.java
deleted file mode 100644
index 12bc35c..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestGlobalEventLoopGroup.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/**
- * 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.ipc;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNotSame;
-import static org.junit.Assert.assertSame;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.testclassification.RPCTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-@Category({ RPCTests.class, SmallTests.class })
-public class TestGlobalEventLoopGroup {
-
-  @Test
-  public void test() {
-    Configuration conf = HBaseConfiguration.create();
-    conf.setBoolean(AsyncRpcClient.USE_GLOBAL_EVENT_LOOP_GROUP, true);
-    AsyncRpcClient client = new AsyncRpcClient(conf);
-    assertNotNull(AsyncRpcClient.GLOBAL_EVENT_LOOP_GROUP);
-    AsyncRpcClient client1 = new AsyncRpcClient(conf);
-    assertSame(client.bootstrap.group(), client1.bootstrap.group());
-    client1.close();
-    assertFalse(client.bootstrap.group().isShuttingDown());
-
-    conf.setBoolean(AsyncRpcClient.USE_GLOBAL_EVENT_LOOP_GROUP, false);
-    AsyncRpcClient client2 = new AsyncRpcClient(conf);
-    assertNotSame(client.bootstrap.group(), client2.bootstrap.group());
-    client2.close();
-
-    client.close();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestIPC.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestIPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestIPC.java
deleted file mode 100644
index b88cb7a..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestIPC.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/**
- *
- * 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.ipc;
-
-import static org.mockito.Matchers.anyInt;
-import static org.mockito.Mockito.doThrow;
-import static org.mockito.Mockito.spy;
-
-import java.io.IOException;
-import java.net.Socket;
-
-import javax.net.SocketFactory;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.codec.Codec;
-import org.apache.hadoop.hbase.testclassification.RPCTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.apache.hadoop.net.NetUtils;
-import org.junit.experimental.categories.Category;
-import org.mockito.Mockito;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-@Category({ RPCTests.class, SmallTests.class })
-public class TestIPC extends AbstractTestIPC {
-
-  @Override
-  protected RpcClientImpl createRpcClientNoCodec(Configuration conf) {
-    return new RpcClientImpl(conf, HConstants.CLUSTER_ID_DEFAULT) {
-      @Override
-      Codec getCodec() {
-        return null;
-      }
-    };
-  }
-
-  @Override
-  protected RpcClientImpl createRpcClient(Configuration conf) {
-    return new RpcClientImpl(conf, HConstants.CLUSTER_ID_DEFAULT);
-  }
-
-  @Override
-  protected RpcClientImpl createRpcClientRTEDuringConnectionSetup(Configuration conf)
-      throws IOException {
-    SocketFactory spyFactory = spy(NetUtils.getDefaultSocketFactory(conf));
-    Mockito.doAnswer(new Answer<Socket>() {
-      @Override
-      public Socket answer(InvocationOnMock invocation) throws Throwable {
-        Socket s = spy((Socket) invocation.callRealMethod());
-        doThrow(new RuntimeException("Injected fault")).when(s).setSoTimeout(anyInt());
-        return s;
-      }
-    }).when(spyFactory).createSocket();
-
-    return new RpcClientImpl(conf, HConstants.CLUSTER_ID_DEFAULT, spyFactory);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestNettyIPC.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestNettyIPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestNettyIPC.java
new file mode 100644
index 0000000..3b32383
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestNettyIPC.java
@@ -0,0 +1,128 @@
+/**
+ * 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.ipc;
+
+import io.netty.channel.epoll.EpollEventLoopGroup;
+import io.netty.channel.epoll.EpollSocketChannel;
+import io.netty.channel.nio.NioEventLoopGroup;
+import io.netty.channel.socket.nio.NioSocketChannel;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.codec.Codec;
+import org.apache.hadoop.hbase.testclassification.RPCTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.JVM;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameter;
+import org.junit.runners.Parameterized.Parameters;
+
+@RunWith(Parameterized.class)
+@Category({ RPCTests.class, SmallTests.class })
+public class TestNettyIPC extends AbstractTestIPC {
+
+  @Parameters(name = "{index}: EventLoop={0}")
+  public static Collection<Object[]> parameters() {
+    List<Object[]> params = new ArrayList<>();
+    params.add(new Object[] { "nio" });
+    params.add(new Object[] { "perClientNio" });
+    if (JVM.isLinux() && JVM.isAmd64()) {
+      params.add(new Object[] { "epoll" });
+    }
+    return params;
+  }
+
+  @Parameter
+  public String eventLoopType;
+
+  private static NioEventLoopGroup NIO;
+
+  private static EpollEventLoopGroup EPOLL;
+
+  @BeforeClass
+  public static void setUpBeforeClass() {
+    NIO = new NioEventLoopGroup();
+    if (JVM.isLinux() && JVM.isAmd64()) {
+      EPOLL = new EpollEventLoopGroup();
+    }
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() {
+    if (NIO != null) {
+      NIO.shutdownGracefully();
+    }
+    if (EPOLL != null) {
+      EPOLL.shutdownGracefully();
+    }
+  }
+
+  private void setConf(Configuration conf) {
+    switch (eventLoopType) {
+      case "nio":
+        NettyRpcClientConfigHelper.setEventLoopConfig(conf, NIO, NioSocketChannel.class);
+        break;
+      case "epoll":
+        NettyRpcClientConfigHelper.setEventLoopConfig(conf, EPOLL, EpollSocketChannel.class);
+        break;
+      case "perClientNio":
+        NettyRpcClientConfigHelper.createEventLoopPerClient(conf);
+        break;
+      default:
+        break;
+    }
+  }
+
+  @Override
+  protected NettyRpcClient createRpcClientNoCodec(Configuration conf) {
+    setConf(conf);
+    return new NettyRpcClient(conf) {
+
+      @Override
+      Codec getCodec() {
+        return null;
+      }
+
+    };
+  }
+
+  @Override
+  protected NettyRpcClient createRpcClient(Configuration conf) {
+    setConf(conf);
+    return new NettyRpcClient(conf);
+  }
+
+  @Override
+  protected NettyRpcClient createRpcClientRTEDuringConnectionSetup(Configuration conf) {
+    setConf(conf);
+    return new NettyRpcClient(conf) {
+
+      @Override
+      boolean isTcpNoDelay() {
+        throw new RuntimeException("Injected fault");
+      }
+    };
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestProtobufRpcServiceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestProtobufRpcServiceImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestProtobufRpcServiceImpl.java
index ae658a3..6354123 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestProtobufRpcServiceImpl.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestProtobufRpcServiceImpl.java
@@ -62,7 +62,7 @@ public class TestProtobufRpcServiceImpl implements BlockingInterface {
   }
 
   public static Interface newStub(RpcClient client, InetSocketAddress addr) throws IOException {
-    return TestProtobufRpcProto.newStub(client.createProtobufRpcChannel(
+    return TestProtobufRpcProto.newStub(client.createRpcChannel(
       ServerName.valueOf(addr.getHostName(), addr.getPort(), System.currentTimeMillis()),
       User.getCurrent(), 0));
   }


[5/6] hbase git commit: HBASE-16445 Refactor and reimplement RpcClient

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcClient.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcClient.java
deleted file mode 100644
index 5f4d2f4..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcClient.java
+++ /dev/null
@@ -1,510 +0,0 @@
-/**
- * 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.ipc;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.protobuf.Descriptors;
-import com.google.protobuf.Message;
-import com.google.protobuf.RpcCallback;
-import com.google.protobuf.RpcChannel;
-import com.google.protobuf.RpcController;
-
-import io.netty.bootstrap.Bootstrap;
-import io.netty.channel.Channel;
-import io.netty.channel.ChannelInitializer;
-import io.netty.channel.ChannelOption;
-import io.netty.channel.EventLoopGroup;
-import io.netty.channel.epoll.EpollEventLoopGroup;
-import io.netty.channel.epoll.EpollSocketChannel;
-import io.netty.channel.nio.NioEventLoopGroup;
-import io.netty.channel.socket.SocketChannel;
-import io.netty.channel.socket.nio.NioSocketChannel;
-import io.netty.util.HashedWheelTimer;
-import io.netty.util.Timeout;
-import io.netty.util.TimerTask;
-import io.netty.util.concurrent.Future;
-import io.netty.util.concurrent.FutureListener;
-import io.netty.util.concurrent.Promise;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.net.SocketAddress;
-import java.nio.ByteBuffer;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.CellScanner;
-import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.client.MetricsConnection;
-import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.util.JVM;
-import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.hbase.util.PoolMap;
-import org.apache.hadoop.hbase.util.Threads;
-
-/**
- * Netty client for the requests and responses
- */
-@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
-public class AsyncRpcClient extends AbstractRpcClient {
-
-  private static final Log LOG = LogFactory.getLog(AsyncRpcClient.class);
-
-  public static final String CLIENT_MAX_THREADS = "hbase.rpc.client.threads.max";
-  public static final String USE_NATIVE_TRANSPORT = "hbase.rpc.client.nativetransport";
-  public static final String USE_GLOBAL_EVENT_LOOP_GROUP = "hbase.rpc.client.globaleventloopgroup";
-
-  private static final HashedWheelTimer WHEEL_TIMER =
-      new HashedWheelTimer(Threads.newDaemonThreadFactory("AsyncRpcChannel-timer"),
-          100, TimeUnit.MILLISECONDS);
-
-  private static final ChannelInitializer<SocketChannel> DEFAULT_CHANNEL_INITIALIZER =
-      new ChannelInitializer<SocketChannel>() {
-    @Override
-    protected void initChannel(SocketChannel ch) throws Exception {
-      //empty initializer
-    }
-  };
-
-  protected final AtomicInteger callIdCnt = new AtomicInteger();
-
-  private final PoolMap<Integer, AsyncRpcChannel> connections;
-
-  final FailedServers failedServers;
-
-  @VisibleForTesting
-  final Bootstrap bootstrap;
-
-  private final boolean useGlobalEventLoopGroup;
-
-  @VisibleForTesting
-  static Pair<EventLoopGroup, Class<? extends Channel>> GLOBAL_EVENT_LOOP_GROUP;
-
-  synchronized static Pair<EventLoopGroup, Class<? extends Channel>>
-      getGlobalEventLoopGroup(Configuration conf) {
-    if (GLOBAL_EVENT_LOOP_GROUP == null) {
-      GLOBAL_EVENT_LOOP_GROUP = createEventLoopGroup(conf);
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Create global event loop group "
-            + GLOBAL_EVENT_LOOP_GROUP.getFirst().getClass().getSimpleName());
-      }
-    }
-    return GLOBAL_EVENT_LOOP_GROUP;
-  }
-
-  private static Pair<EventLoopGroup, Class<? extends Channel>> createEventLoopGroup(
-      Configuration conf) {
-    // Max amount of threads to use. 0 lets Netty decide based on amount of cores
-    int maxThreads = conf.getInt(CLIENT_MAX_THREADS, 0);
-
-    // Config to enable native transport. Does not seem to be stable at time of implementation
-    // although it is not extensively tested.
-    boolean epollEnabled = conf.getBoolean(USE_NATIVE_TRANSPORT, false);
-
-    // Use the faster native epoll transport mechanism on linux if enabled
-    if (epollEnabled && JVM.isLinux() && JVM.isAmd64()) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Create EpollEventLoopGroup with maxThreads = " + maxThreads);
-      }
-      return new Pair<EventLoopGroup, Class<? extends Channel>>(new EpollEventLoopGroup(maxThreads,
-          Threads.newDaemonThreadFactory("AsyncRpcChannel")), EpollSocketChannel.class);
-    } else {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Create NioEventLoopGroup with maxThreads = " + maxThreads);
-      }
-      return new Pair<EventLoopGroup, Class<? extends Channel>>(new NioEventLoopGroup(maxThreads,
-          Threads.newDaemonThreadFactory("AsyncRpcChannel")), NioSocketChannel.class);
-    }
-  }
-
-  /**
-   * Constructor for tests
-   *
-   * @param configuration      to HBase
-   * @param clusterId          for the cluster
-   * @param localAddress       local address to connect to
-   * @param metrics            the connection metrics
-   * @param channelInitializer for custom channel handlers
-   */
-  protected AsyncRpcClient(Configuration configuration, String clusterId,
-      SocketAddress localAddress, MetricsConnection metrics,
-      ChannelInitializer<SocketChannel> channelInitializer) {
-    super(configuration, clusterId, localAddress, metrics);
-
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Starting async Hbase RPC client");
-    }
-
-    Pair<EventLoopGroup, Class<? extends Channel>> eventLoopGroupAndChannelClass;
-    this.useGlobalEventLoopGroup = conf.getBoolean(USE_GLOBAL_EVENT_LOOP_GROUP, true);
-    if (useGlobalEventLoopGroup) {
-      eventLoopGroupAndChannelClass = getGlobalEventLoopGroup(configuration);
-    } else {
-      eventLoopGroupAndChannelClass = createEventLoopGroup(configuration);
-    }
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Use " + (useGlobalEventLoopGroup ? "global" : "individual") + " event loop group "
-          + eventLoopGroupAndChannelClass.getFirst().getClass().getSimpleName());
-    }
-
-    this.connections = new PoolMap<>(getPoolType(configuration), getPoolSize(configuration));
-    this.failedServers = new FailedServers(configuration);
-
-    int operationTimeout = configuration.getInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT,
-        HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT);
-
-    // Configure the default bootstrap.
-    this.bootstrap = new Bootstrap();
-    bootstrap.group(eventLoopGroupAndChannelClass.getFirst())
-        .channel(eventLoopGroupAndChannelClass.getSecond())
-        .option(ChannelOption.TCP_NODELAY, tcpNoDelay)
-        .option(ChannelOption.SO_KEEPALIVE, tcpKeepAlive)
-        .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, operationTimeout);
-    if (channelInitializer == null) {
-      channelInitializer = DEFAULT_CHANNEL_INITIALIZER;
-    }
-    bootstrap.handler(channelInitializer);
-    if (localAddress != null) {
-      bootstrap.localAddress(localAddress);
-    }
-  }
-
-  /** Used in test only. */
-  AsyncRpcClient(Configuration configuration) {
-    this(configuration, HConstants.CLUSTER_ID_DEFAULT, null, null);
-  }
-
-  /** Used in test only. */
-  AsyncRpcClient(Configuration configuration,
-      ChannelInitializer<SocketChannel> channelInitializer) {
-    this(configuration, HConstants.CLUSTER_ID_DEFAULT, null, null, channelInitializer);
-  }
-
-  /**
-   * Constructor
-   *
-   * @param configuration to HBase
-   * @param clusterId     for the cluster
-   * @param localAddress  local address to connect to
-   * @param metrics       the connection metrics
-   */
-  public AsyncRpcClient(Configuration configuration, String clusterId, SocketAddress localAddress,
-      MetricsConnection metrics) {
-    this(configuration, clusterId, localAddress, metrics, null);
-  }
-
-  /**
-   * Make a call, passing <code>param</code>, to the IPC server running at
-   * <code>address</code> which is servicing the <code>protocol</code> protocol,
-   * with the <code>ticket</code> credentials, returning the value.
-   * Throws exceptions if there are network problems or if the remote code
-   * threw an exception.
-   *
-   * @param ticket Be careful which ticket you pass. A new user will mean a new Connection.
-   *               {@link org.apache.hadoop.hbase.security.UserProvider#getCurrent()} makes a new
-   *               instance of User each time so will be a new Connection each time.
-   * @return A pair with the Message response and the Cell data (if any).
-   * @throws InterruptedException if call is interrupted
-   * @throws java.io.IOException  if a connection failure is encountered
-   */
-  @Override
-  protected Pair<Message, CellScanner> call(HBaseRpcController pcrc,
-      Descriptors.MethodDescriptor md, Message param, Message returnType, User ticket,
-      InetSocketAddress addr, MetricsConnection.CallStats callStats)
-      throws IOException, InterruptedException {
-    if (pcrc == null) {
-      pcrc = new HBaseRpcControllerImpl();
-    }
-    final AsyncRpcChannel connection = createRpcChannel(md.getService().getName(), addr, ticket);
-
-    final Promise<Message> promise = connection.callMethod(md, param, pcrc.cellScanner(), returnType,
-        getMessageConverterWithRpcController(pcrc), null, pcrc.getCallTimeout(),
-        pcrc.getPriority());
-
-    pcrc.notifyOnCancel(new RpcCallback<Object>() {
-      @Override
-      public void run(Object parameter) {
-        // Will automatically fail the promise with CancellationException
-        promise.cancel(true);
-      }
-    });
-
-    long timeout = pcrc.hasCallTimeout() ? pcrc.getCallTimeout() : 0;
-    try {
-      Message response = timeout > 0 ? promise.get(timeout, TimeUnit.MILLISECONDS) : promise.get();
-      return new Pair<>(response, pcrc.cellScanner());
-    } catch (ExecutionException e) {
-      if (e.getCause() instanceof IOException) {
-        throw (IOException) e.getCause();
-      } else {
-        throw IPCUtil.wrapException(addr, (Exception) e.getCause());
-      }
-    } catch (TimeoutException e) {
-      CallTimeoutException cte = new CallTimeoutException(promise.toString());
-      throw IPCUtil.wrapException(addr, cte);
-    }
-  }
-
-  private MessageConverter<Message, Message> getMessageConverterWithRpcController(
-      final HBaseRpcController pcrc) {
-    return new
-      MessageConverter<Message, Message>() {
-        @Override
-        public Message convert(Message msg, CellScanner cellScanner) {
-          pcrc.setCellScanner(cellScanner);
-          return msg;
-        }
-      };
-  }
-
-  /**
-   * Call method async
-   */
-  private void callMethod(final Descriptors.MethodDescriptor md,
-      final HBaseRpcController pcrc, final Message param, Message returnType, User ticket,
-      InetSocketAddress addr, final RpcCallback<Message> done) {
-    final AsyncRpcChannel connection;
-    try {
-      connection = createRpcChannel(md.getService().getName(), addr, ticket);
-
-      FutureListener<Message> listener =
-        new FutureListener<Message>() {
-          @Override
-          public void operationComplete(Future<Message> future) throws Exception {
-            if (!future.isSuccess()) {
-              Throwable cause = future.cause();
-              if (cause instanceof IOException) {
-                pcrc.setFailed((IOException) cause);
-              } else {
-                pcrc.setFailed(new IOException(cause));
-              }
-            } else {
-              try {
-                done.run(future.get());
-              } catch (ExecutionException e) {
-                Throwable cause = e.getCause();
-                if (cause instanceof IOException) {
-                  pcrc.setFailed((IOException) cause);
-                } else {
-                  pcrc.setFailed(new IOException(cause));
-                }
-              } catch (InterruptedException e) {
-                pcrc.setFailed(new IOException(e));
-              }
-            }
-          }
-        };
-      connection.callMethod(md, param, pcrc.cellScanner(), returnType,
-          getMessageConverterWithRpcController(pcrc), null,
-          pcrc.getCallTimeout(), pcrc.getPriority())
-          .addListener(listener);
-    } catch (StoppedRpcClientException|FailedServerException e) {
-      pcrc.setFailed(e);
-    }
-  }
-
-  private boolean closed = false;
-
-  /**
-   * Close netty
-   */
-  public void close() {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Stopping async HBase RPC client");
-    }
-
-    synchronized (connections) {
-      if (closed) {
-        return;
-      }
-      closed = true;
-      for (AsyncRpcChannel conn : connections.values()) {
-        conn.close(null);
-      }
-    }
-    // do not close global EventLoopGroup.
-    if (!useGlobalEventLoopGroup) {
-      bootstrap.config().group().shutdownGracefully();
-    }
-  }
-
-  /**
-   * Create a cell scanner
-   *
-   * @param cellBlock to create scanner for
-   * @return CellScanner
-   * @throws java.io.IOException on error on creation cell scanner
-   */
-  public CellScanner createCellScanner(byte[] cellBlock) throws IOException {
-    return cellBlockBuilder.createCellScanner(this.codec, this.compressor, cellBlock);
-  }
-
-  /**
-   * Build cell block
-   *
-   * @param cells to create block with
-   * @return ByteBuffer with cells
-   * @throws java.io.IOException if block creation fails
-   */
-  public ByteBuffer buildCellBlock(CellScanner cells) throws IOException {
-    return cellBlockBuilder.buildCellBlock(this.codec, this.compressor, cells);
-  }
-
-  /**
-   * Creates an RPC client
-   *
-   * @param serviceName    name of service
-   * @param location       to connect to
-   * @param ticket         for current user
-   * @return new RpcChannel
-   * @throws StoppedRpcClientException when Rpc client is stopped
-   * @throws FailedServerException if server failed
-   */
-  private AsyncRpcChannel createRpcChannel(String serviceName, InetSocketAddress location,
-      User ticket) throws StoppedRpcClientException, FailedServerException {
-    // Check if server is failed
-    if (this.failedServers.isFailedServer(location)) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Not trying to connect to " + location +
-            " this server is in the failed servers list");
-      }
-      throw new FailedServerException(
-          "This server is in the failed servers list: " + location);
-    }
-
-    int hashCode = ConnectionId.hashCode(ticket,serviceName,location);
-
-    AsyncRpcChannel rpcChannel;
-    synchronized (connections) {
-      if (closed) {
-        throw new StoppedRpcClientException();
-      }
-      rpcChannel = connections.get(hashCode);
-      if (rpcChannel != null && !rpcChannel.isAlive()) {
-        LOG.debug("Removing dead channel from server="+rpcChannel.getAddress().toString());
-        connections.remove(hashCode);
-      }
-      if (rpcChannel == null) {
-        rpcChannel = new AsyncRpcChannel(this.bootstrap, this, ticket, serviceName, location);
-        connections.put(hashCode, rpcChannel);
-      }
-    }
-
-    return rpcChannel;
-  }
-
-  /**
-   * Interrupt the connections to the given ip:port server. This should be called if the server
-   * is known as actually dead. This will not prevent current operation to be retried, and,
-   * depending on their own behavior, they may retry on the same server. This can be a feature,
-   * for example at startup. In any case, they're likely to get connection refused (if the
-   * process died) or no route to host: i.e. there next retries should be faster and with a
-   * safe exception.
-   *
-   * @param sn server to cancel connections for
-   */
-  @Override
-  public void cancelConnections(ServerName sn) {
-    synchronized (connections) {
-      for (AsyncRpcChannel rpcChannel : connections.values()) {
-        if (rpcChannel.isAlive() &&
-            rpcChannel.getAddress().getPort() == sn.getPort() &&
-            rpcChannel.getAddress().getHostName().contentEquals(sn.getHostname())) {
-          LOG.info("The server on " + sn.toString() +
-              " is dead - stopping the connection " + rpcChannel.toString());
-          rpcChannel.close(null);
-        }
-      }
-    }
-  }
-
-  /**
-   * Remove connection from pool
-   * @param connection to remove
-   */
-  public void removeConnection(AsyncRpcChannel connection) {
-    int connectionHashCode = connection.hashCode();
-    synchronized (connections) {
-      // we use address as cache key, so we should check here to prevent removing the
-      // wrong connection
-      AsyncRpcChannel connectionInPool = this.connections.get(connectionHashCode);
-      if (connectionInPool != null && connectionInPool.equals(connection)) {
-        this.connections.remove(connectionHashCode);
-      } else if (LOG.isDebugEnabled()) {
-        LOG.debug(String.format("%s already removed, expected instance %08x, actual %08x",
-          connection.toString(), System.identityHashCode(connection),
-          System.identityHashCode(connectionInPool)));
-      }
-    }
-  }
-
-  @Override
-  public RpcChannel createProtobufRpcChannel(final ServerName sn, final User user, int rpcTimeout) {
-    return new RpcChannelImplementation(this, sn, user, rpcTimeout);
-  }
-
-  /**
-   * Blocking rpc channel that goes via hbase rpc.
-   */
-  @VisibleForTesting
-  public static class RpcChannelImplementation implements RpcChannel {
-    private final InetSocketAddress isa;
-    private final AsyncRpcClient rpcClient;
-    private final User ticket;
-    private final int channelOperationTimeout;
-
-    /**
-     * @param channelOperationTimeout - the default timeout when no timeout is given
-     */
-    protected RpcChannelImplementation(final AsyncRpcClient rpcClient,
-        final ServerName sn, final User ticket, int channelOperationTimeout) {
-      this.isa = new InetSocketAddress(sn.getHostname(), sn.getPort());
-      this.rpcClient = rpcClient;
-      this.ticket = ticket;
-      this.channelOperationTimeout = channelOperationTimeout;
-    }
-
-    @Override
-    public void callMethod(Descriptors.MethodDescriptor md, RpcController controller,
-        Message param, Message returnType, RpcCallback<Message> done) {
-      HBaseRpcController pcrc =
-          configurePayloadCarryingRpcController(controller, channelOperationTimeout);
-
-      this.rpcClient.callMethod(md, pcrc, param, returnType, this.ticket, this.isa, done);
-    }
-  }
-
-  /**
-   * Get a new timeout on this RPC client
-   * @param task to run at timeout
-   * @param delay for the timeout
-   * @param unit time unit for the timeout
-   * @return Timeout
-   */
-  Timeout newTimeout(TimerTask task, long delay, TimeUnit unit) {
-    return WHEEL_TIMER.newTimeout(task, delay, unit);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncServerResponseHandler.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncServerResponseHandler.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncServerResponseHandler.java
deleted file mode 100644
index 7a2802f..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncServerResponseHandler.java
+++ /dev/null
@@ -1,126 +0,0 @@
-/**
- * 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.ipc;
-
-import com.google.protobuf.Message;
-
-import io.netty.buffer.ByteBuf;
-import io.netty.buffer.ByteBufInputStream;
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.SimpleChannelInboundHandler;
-
-import java.io.IOException;
-
-import org.apache.hadoop.hbase.CellScanner;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.protobuf.generated.RPCProtos;
-import org.apache.hadoop.ipc.RemoteException;
-
-/**
- * Handles Hbase responses
- */
-@InterfaceAudience.Private
-public class AsyncServerResponseHandler extends SimpleChannelInboundHandler<ByteBuf> {
-  private final AsyncRpcChannel channel;
-
-  /**
-   * Constructor
-   * @param channel on which this response handler operates
-   */
-  public AsyncServerResponseHandler(AsyncRpcChannel channel) {
-    this.channel = channel;
-  }
-
-  @Override
-  protected void channelRead0(ChannelHandlerContext ctx, ByteBuf inBuffer) throws Exception {
-    ByteBufInputStream in = new ByteBufInputStream(inBuffer);
-    int totalSize = inBuffer.readableBytes();
-    // Read the header
-    RPCProtos.ResponseHeader responseHeader = RPCProtos.ResponseHeader.parseDelimitedFrom(in);
-    int id = responseHeader.getCallId();
-    AsyncCall call = channel.removePendingCall(id);
-    if (call == null) {
-      // So we got a response for which we have no corresponding 'call' here on the client-side.
-      // We probably timed out waiting, cleaned up all references, and now the server decides
-      // to return a response. There is nothing we can do w/ the response at this stage. Clean
-      // out the wire of the response so its out of the way and we can get other responses on
-      // this connection.
-      int readSoFar = IPCUtil.getTotalSizeWhenWrittenDelimited(responseHeader);
-      int whatIsLeftToRead = totalSize - readSoFar;
-
-      // This is done through a Netty ByteBuf which has different behavior than InputStream.
-      // It does not return number of bytes read but will update pointer internally and throws an
-      // exception when too many bytes are to be skipped.
-      inBuffer.skipBytes(whatIsLeftToRead);
-      return;
-    }
-
-    if (responseHeader.hasException()) {
-      RPCProtos.ExceptionResponse exceptionResponse = responseHeader.getException();
-      RemoteException re = createRemoteException(exceptionResponse);
-      if (exceptionResponse.getExceptionClassName()
-          .equals(FatalConnectionException.class.getName())) {
-        channel.close(re);
-      } else {
-        call.setFailed(re);
-      }
-    } else {
-      Message value = null;
-      // Call may be null because it may have timedout and been cleaned up on this side already
-      if (call.responseDefaultType != null) {
-        Message.Builder builder = call.responseDefaultType.newBuilderForType();
-        ProtobufUtil.mergeDelimitedFrom(builder, in);
-        value = builder.build();
-      }
-      CellScanner cellBlockScanner = null;
-      if (responseHeader.hasCellBlockMeta()) {
-        int size = responseHeader.getCellBlockMeta().getLength();
-        byte[] cellBlock = new byte[size];
-        inBuffer.readBytes(cellBlock, 0, cellBlock.length);
-        cellBlockScanner = channel.client.createCellScanner(cellBlock);
-      }
-      call.setSuccess(value, cellBlockScanner);
-      call.callStats.setResponseSizeBytes(totalSize);
-    }
-  }
-
-  @Override
-  public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception {
-    channel.close(cause);
-  }
-
-  @Override
-  public void channelInactive(ChannelHandlerContext ctx) throws Exception {
-    channel.close(new IOException("connection closed"));
-  }
-
-  /**
-   * @param e Proto exception
-   * @return RemoteException made from passed <code>e</code>
-   */
-  private RemoteException createRemoteException(final RPCProtos.ExceptionResponse e) {
-    String innerExceptionClassName = e.getExceptionClassName();
-    boolean doNotRetry = e.getDoNotRetry();
-    return e.hasHostname() ?
-        // If a hostname then add it to the RemoteWithExtrasException
-        new RemoteWithExtrasException(innerExceptionClassName, e.getStackTrace(), e.getHostname(),
-            e.getPort(), doNotRetry)
-        : new RemoteWithExtrasException(innerExceptionClassName, e.getStackTrace(), doNotRetry);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcCallback.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcCallback.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcCallback.java
index 0475e58..523ca55 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcCallback.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcCallback.java
@@ -46,7 +46,7 @@ public class BlockingRpcCallback<R> implements RpcCallback<R> {
     synchronized (this) {
       result = parameter;
       resultSet = true;
-      this.notify();
+      this.notifyAll();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcClient.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcClient.java
new file mode 100644
index 0000000..d27602e
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcClient.java
@@ -0,0 +1,77 @@
+/**
+ * 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.ipc;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import java.io.IOException;
+import java.net.SocketAddress;
+
+import javax.net.SocketFactory;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.MetricsConnection;
+import org.apache.hadoop.net.NetUtils;
+
+/**
+ * Does RPC against a cluster. Manages connections per regionserver in the cluster.
+ * <p>
+ * See HBaseServer
+ */
+@InterfaceAudience.Private
+public class BlockingRpcClient extends AbstractRpcClient<BlockingRpcConnection> {
+
+  protected final SocketFactory socketFactory; // how to create sockets
+
+  /**
+   * Used in test only. Construct an IPC client for the cluster {@code clusterId} with the default
+   * SocketFactory
+   */
+  @VisibleForTesting
+  BlockingRpcClient(Configuration conf) {
+    this(conf, HConstants.CLUSTER_ID_DEFAULT, null, null);
+  }
+
+  /**
+   * Construct an IPC client for the cluster {@code clusterId} with the default SocketFactory This
+   * method is called with reflection by the RpcClientFactory to create an instance
+   * @param conf configuration
+   * @param clusterId the cluster id
+   * @param localAddr client socket bind address.
+   * @param metrics the connection metrics
+   */
+  public BlockingRpcClient(Configuration conf, String clusterId, SocketAddress localAddr,
+      MetricsConnection metrics) {
+    super(conf, clusterId, localAddr, metrics);
+    this.socketFactory = NetUtils.getDefaultSocketFactory(conf);
+  }
+
+  /**
+   * Creates a connection. Can be overridden by a subclass for testing.
+   * @param remoteId - the ConnectionId to use for the connection creation.
+   */
+  protected BlockingRpcConnection createConnection(ConnectionId remoteId) throws IOException {
+    return new BlockingRpcConnection(this, remoteId);
+  }
+
+  @Override
+  protected void closeInternal() {
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcConnection.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcConnection.java
new file mode 100644
index 0000000..c8b366d
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcConnection.java
@@ -0,0 +1,725 @@
+/**
+ * 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.ipc;
+
+import static org.apache.hadoop.hbase.ipc.IPCUtil.buildRequestHeader;
+import static org.apache.hadoop.hbase.ipc.IPCUtil.createRemoteException;
+import static org.apache.hadoop.hbase.ipc.IPCUtil.getTotalSizeWhenWrittenDelimited;
+import static org.apache.hadoop.hbase.ipc.IPCUtil.isFatalConnectionException;
+import static org.apache.hadoop.hbase.ipc.IPCUtil.setCancelled;
+import static org.apache.hadoop.hbase.ipc.IPCUtil.write;
+
+import com.google.protobuf.Message;
+import com.google.protobuf.Message.Builder;
+import com.google.protobuf.RpcCallback;
+
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InterruptedIOException;
+import java.io.OutputStream;
+import java.net.Socket;
+import java.net.SocketTimeoutException;
+import java.net.UnknownHostException;
+import java.nio.ByteBuffer;
+import java.security.PrivilegedExceptionAction;
+import java.util.ArrayDeque;
+import java.util.Locale;
+import java.util.Queue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ThreadLocalRandom;
+
+import javax.security.sasl.SaslException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.CellScanner;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.exceptions.ConnectionClosingException;
+import org.apache.hadoop.hbase.io.ByteArrayOutputStream;
+import org.apache.hadoop.hbase.ipc.HBaseRpcController.CancellationCallback;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta;
+import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ConnectionHeader;
+import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse;
+import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader;
+import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader;
+import org.apache.hadoop.hbase.security.HBaseSaslRpcClient;
+import org.apache.hadoop.hbase.security.SaslUtil.QualityOfProtection;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.ExceptionUtil;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.htrace.Trace;
+import org.apache.htrace.TraceScope;
+
+/**
+ * Thread that reads responses and notifies callers. Each connection owns a socket connected to a
+ * remote address. Calls are multiplexed through this socket: responses may be delivered out of
+ * order.
+ */
+@InterfaceAudience.Private
+class BlockingRpcConnection extends RpcConnection implements Runnable {
+
+  private static final Log LOG = LogFactory.getLog(BlockingRpcConnection.class);
+
+  private final BlockingRpcClient rpcClient;
+
+  private final String threadName;
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "IS2_INCONSISTENT_SYNC",
+      justification = "We are always under lock actually")
+  private Thread thread;
+
+  // connected socket. protected for writing UT.
+  protected Socket socket = null;
+  private DataInputStream in;
+  private DataOutputStream out;
+
+  private HBaseSaslRpcClient saslRpcClient;
+
+  // currently active calls
+  private final ConcurrentMap<Integer, Call> calls = new ConcurrentHashMap<>();
+
+  private final CallSender callSender;
+
+  private boolean closed = false;
+
+  private byte[] connectionHeaderPreamble;
+
+  private byte[] connectionHeaderWithLength;
+
+  /**
+   * If the client wants to interrupt its calls easily (i.e. call Thread#interrupt), it gets into a
+   * java issue: an interruption during a write closes the socket/channel. A way to avoid this is to
+   * use a different thread for writing. This way, on interruptions, we either cancel the writes or
+   * ignore the answer if the write is already done, but we don't stop the write in the middle. This
+   * adds a thread per region server in the client, so it's kept as an option.
+   * <p>
+   * The implementation is simple: the client threads adds their call to the queue, and then wait
+   * for an answer. The CallSender blocks on the queue, and writes the calls one after the other. On
+   * interruption, the client cancels its call. The CallSender checks that the call has not been
+   * canceled before writing it.
+   * </p>
+   * When the connection closes, all the calls not yet sent are dismissed. The client thread is
+   * notified with an appropriate exception, as if the call was already sent but the answer not yet
+   * received.
+   * </p>
+   */
+  private class CallSender extends Thread {
+
+    private final Queue<Call> callsToWrite;
+
+    private final int maxQueueSize;
+
+    public CallSender(String name, Configuration conf) {
+      int queueSize = conf.getInt("hbase.ipc.client.write.queueSize", 1000);
+      callsToWrite = new ArrayDeque<>(queueSize);
+      this.maxQueueSize = queueSize;
+      setDaemon(true);
+      setName(name + " - writer");
+    }
+
+    public void sendCall(final Call call) throws IOException {
+      if (callsToWrite.size() >= maxQueueSize) {
+        throw new IOException("Can't add the call " + call.id
+            + " to the write queue. callsToWrite.size()=" + callsToWrite.size());
+      }
+      callsToWrite.offer(call);
+      BlockingRpcConnection.this.notifyAll();
+    }
+
+    public void remove(Call call) {
+      callsToWrite.remove();
+      // By removing the call from the expected call list, we make the list smaller, but
+      // it means as well that we don't know how many calls we cancelled.
+      calls.remove(call.id);
+      call.setException(new CallCancelledException("Call id=" + call.id + ", waitTime="
+          + (EnvironmentEdgeManager.currentTime() - call.getStartTime()) + ", rpcTimeout="
+          + call.timeout));
+    }
+
+    /**
+     * Reads the call from the queue, write them on the socket.
+     */
+    @Override
+    public void run() {
+      synchronized (BlockingRpcConnection.this) {
+        while (!closed) {
+          if (callsToWrite.isEmpty()) {
+            // We should use another monitor object here for better performance since the read
+            // thread also uses ConnectionImpl.this. But this makes the locking schema more
+            // complicated, can do it later as an optimization.
+            try {
+              BlockingRpcConnection.this.wait();
+            } catch (InterruptedException e) {
+            }
+            // check if we need to quit, so continue the main loop instead of fallback.
+            continue;
+          }
+          Call call = callsToWrite.poll();
+          if (call.isDone()) {
+            continue;
+          }
+          try {
+            tracedWriteRequest(call);
+          } catch (IOException e) {
+            // exception here means the call has not been added to the pendingCalls yet, so we need
+            // to fail it by our own.
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("call write error for call #" + call.id, e);
+            }
+            call.setException(e);
+            closeConn(e);
+          }
+        }
+      }
+    }
+
+    /**
+     * Cleans the call not yet sent when we finish.
+     */
+    public void cleanup(IOException e) {
+      IOException ie = new ConnectionClosingException(
+          "Connection to " + remoteId.address + " is closing.");
+      for (Call call : callsToWrite) {
+        call.setException(ie);
+      }
+      callsToWrite.clear();
+    }
+  }
+
+  BlockingRpcConnection(BlockingRpcClient rpcClient, ConnectionId remoteId) throws IOException {
+    super(rpcClient.conf, AbstractRpcClient.WHEEL_TIMER, remoteId, rpcClient.clusterId,
+        rpcClient.userProvider.isHBaseSecurityEnabled(), rpcClient.codec, rpcClient.compressor);
+    this.rpcClient = rpcClient;
+    if (remoteId.getAddress().isUnresolved()) {
+      throw new UnknownHostException("unknown host: " + remoteId.getAddress().getHostName());
+    }
+
+    this.connectionHeaderPreamble = getConnectionHeaderPreamble();
+    ConnectionHeader header = getConnectionHeader();
+    ByteArrayOutputStream baos = new ByteArrayOutputStream(4 + header.getSerializedSize());
+    DataOutputStream dos = new DataOutputStream(baos);
+    dos.writeInt(header.getSerializedSize());
+    header.writeTo(dos);
+    assert baos.size() == 4 + header.getSerializedSize();
+    this.connectionHeaderWithLength = baos.getBuffer();
+
+    UserGroupInformation ticket = remoteId.ticket.getUGI();
+    this.threadName = "IPC Client (" + this.rpcClient.socketFactory.hashCode() + ") connection to "
+        + remoteId.getAddress().toString()
+        + ((ticket == null) ? " from an unknown user" : (" from " + ticket.getUserName()));
+
+    if (this.rpcClient.conf.getBoolean(BlockingRpcClient.SPECIFIC_WRITE_THREAD, false)) {
+      callSender = new CallSender(threadName, this.rpcClient.conf);
+      callSender.start();
+    } else {
+      callSender = null;
+    }
+  }
+
+  // protected for write UT.
+  protected void setupConnection() throws IOException {
+    short ioFailures = 0;
+    short timeoutFailures = 0;
+    while (true) {
+      try {
+        this.socket = this.rpcClient.socketFactory.createSocket();
+        this.socket.setTcpNoDelay(this.rpcClient.isTcpNoDelay());
+        this.socket.setKeepAlive(this.rpcClient.tcpKeepAlive);
+        if (this.rpcClient.localAddr != null) {
+          this.socket.bind(this.rpcClient.localAddr);
+        }
+        NetUtils.connect(this.socket, remoteId.getAddress(), this.rpcClient.connectTO);
+        this.socket.setSoTimeout(this.rpcClient.readTO);
+        return;
+      } catch (SocketTimeoutException toe) {
+        /*
+         * The max number of retries is 45, which amounts to 20s*45 = 15 minutes retries.
+         */
+        handleConnectionFailure(timeoutFailures++, this.rpcClient.maxRetries, toe);
+      } catch (IOException ie) {
+        handleConnectionFailure(ioFailures++, this.rpcClient.maxRetries, ie);
+      }
+    }
+  }
+
+  /**
+   * Handle connection failures If the current number of retries is equal to the max number of
+   * retries, stop retrying and throw the exception; Otherwise backoff N seconds and try connecting
+   * again. This Method is only called from inside setupIOstreams(), which is synchronized. Hence
+   * the sleep is synchronized; the locks will be retained.
+   * @param curRetries current number of retries
+   * @param maxRetries max number of retries allowed
+   * @param ioe failure reason
+   * @throws IOException if max number of retries is reached
+   */
+  private void handleConnectionFailure(int curRetries, int maxRetries, IOException ioe)
+      throws IOException {
+    closeSocket();
+
+    // throw the exception if the maximum number of retries is reached
+    if (curRetries >= maxRetries || ExceptionUtil.isInterrupt(ioe)) {
+      throw ioe;
+    }
+
+    // otherwise back off and retry
+    try {
+      Thread.sleep(this.rpcClient.failureSleep);
+    } catch (InterruptedException ie) {
+      ExceptionUtil.rethrowIfInterrupt(ie);
+    }
+
+    LOG.info("Retrying connect to server: " + remoteId.getAddress() + " after sleeping "
+        + this.rpcClient.failureSleep + "ms. Already tried " + curRetries + " time(s).");
+  }
+
+  /*
+   * wait till someone signals us to start reading RPC response or it is idle too long, it is marked
+   * as to be closed, or the client is marked as not running.
+   * @return true if it is time to read a response; false otherwise.
+   */
+  private synchronized boolean waitForWork() {
+    // beware of the concurrent access to the calls list: we can add calls, but as well
+    // remove them.
+    long waitUntil = EnvironmentEdgeManager.currentTime() + this.rpcClient.minIdleTimeBeforeClose;
+    for (;;) {
+      if (thread == null) {
+        return false;
+      }
+      if (!calls.isEmpty()) {
+        return true;
+      }
+      if (EnvironmentEdgeManager.currentTime() >= waitUntil) {
+        closeConn(
+          new IOException("idle connection closed with " + calls.size() + " pending request(s)"));
+        return false;
+      }
+      try {
+        wait(Math.min(this.rpcClient.minIdleTimeBeforeClose, 1000));
+      } catch (InterruptedException e) {
+      }
+    }
+  }
+
+  @Override
+  public void run() {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace(threadName + ": starting, connections " + this.rpcClient.connections.size());
+    }
+    while (waitForWork()) {
+      readResponse();
+    }
+    if (LOG.isTraceEnabled()) {
+      LOG.trace(threadName + ": stopped, connections " + this.rpcClient.connections.size());
+    }
+  }
+
+  private void disposeSasl() {
+    if (saslRpcClient != null) {
+      saslRpcClient.dispose();
+      saslRpcClient = null;
+    }
+  }
+
+  private boolean setupSaslConnection(final InputStream in2, final OutputStream out2)
+      throws IOException {
+    saslRpcClient = new HBaseSaslRpcClient(authMethod, token, serverPrincipal,
+        this.rpcClient.fallbackAllowed, this.rpcClient.conf.get("hbase.rpc.protection",
+          QualityOfProtection.AUTHENTICATION.name().toLowerCase(Locale.ROOT)));
+    return saslRpcClient.saslConnect(in2, out2);
+  }
+
+  /**
+   * If multiple clients with the same principal try to connect to the same server at the same time,
+   * the server assumes a replay attack is in progress. This is a feature of kerberos. In order to
+   * work around this, what is done is that the client backs off randomly and tries to initiate the
+   * connection again. The other problem is to do with ticket expiry. To handle that, a relogin is
+   * attempted.
+   * <p>
+   * The retry logic is governed by the {@link #shouldAuthenticateOverKrb} method. In case when the
+   * user doesn't have valid credentials, we don't need to retry (from cache or ticket). In such
+   * cases, it is prudent to throw a runtime exception when we receive a SaslException from the
+   * underlying authentication implementation, so there is no retry from other high level (for eg,
+   * HCM or HBaseAdmin).
+   * </p>
+   */
+  private void handleSaslConnectionFailure(final int currRetries, final int maxRetries,
+      final Exception ex, final UserGroupInformation user)
+      throws IOException, InterruptedException {
+    closeSocket();
+    user.doAs(new PrivilegedExceptionAction<Object>() {
+      @Override
+      public Object run() throws IOException, InterruptedException {
+        if (shouldAuthenticateOverKrb()) {
+          if (currRetries < maxRetries) {
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("Exception encountered while connecting to " + "the server : " + ex);
+            }
+            // try re-login
+            relogin();
+            disposeSasl();
+            // have granularity of milliseconds
+            // we are sleeping with the Connection lock held but since this
+            // connection instance is being used for connecting to the server
+            // in question, it is okay
+            Thread.sleep(ThreadLocalRandom.current().nextInt(reloginMaxBackoff) + 1);
+            return null;
+          } else {
+            String msg = "Couldn't setup connection for "
+                + UserGroupInformation.getLoginUser().getUserName() + " to " + serverPrincipal;
+            LOG.warn(msg, ex);
+            throw (IOException) new IOException(msg).initCause(ex);
+          }
+        } else {
+          LOG.warn("Exception encountered while connecting to " + "the server : " + ex);
+        }
+        if (ex instanceof RemoteException) {
+          throw (RemoteException) ex;
+        }
+        if (ex instanceof SaslException) {
+          String msg = "SASL authentication failed."
+              + " The most likely cause is missing or invalid credentials." + " Consider 'kinit'.";
+          LOG.fatal(msg, ex);
+          throw new RuntimeException(msg, ex);
+        }
+        throw new IOException(ex);
+      }
+    });
+  }
+
+  private void setupIOstreams() throws IOException {
+    if (socket != null) {
+      // The connection is already available. Perfect.
+      return;
+    }
+
+    if (this.rpcClient.failedServers.isFailedServer(remoteId.getAddress())) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Not trying to connect to " + remoteId.address
+            + " this server is in the failed servers list");
+      }
+      throw new FailedServerException(
+          "This server is in the failed servers list: " + remoteId.address);
+    }
+
+    try {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Connecting to " + remoteId.address);
+      }
+
+      short numRetries = 0;
+      final short MAX_RETRIES = 5;
+      while (true) {
+        setupConnection();
+        InputStream inStream = NetUtils.getInputStream(socket);
+        // This creates a socket with a write timeout. This timeout cannot be changed.
+        OutputStream outStream = NetUtils.getOutputStream(socket, this.rpcClient.writeTO);
+        // Write out the preamble -- MAGIC, version, and auth to use.
+        writeConnectionHeaderPreamble(outStream);
+        if (useSasl) {
+          final InputStream in2 = inStream;
+          final OutputStream out2 = outStream;
+          UserGroupInformation ticket = getUGI();
+          boolean continueSasl;
+          if (ticket == null) {
+            throw new FatalConnectionException("ticket/user is null");
+          }
+          try {
+            continueSasl = ticket.doAs(new PrivilegedExceptionAction<Boolean>() {
+              @Override
+              public Boolean run() throws IOException {
+                return setupSaslConnection(in2, out2);
+              }
+            });
+          } catch (Exception ex) {
+            ExceptionUtil.rethrowIfInterrupt(ex);
+            handleSaslConnectionFailure(numRetries++, MAX_RETRIES, ex, ticket);
+            continue;
+          }
+          if (continueSasl) {
+            // Sasl connect is successful. Let's set up Sasl i/o streams.
+            inStream = saslRpcClient.getInputStream(inStream);
+            outStream = saslRpcClient.getOutputStream(outStream);
+          } else {
+            // fall back to simple auth because server told us so.
+            // do not change authMethod and useSasl here, we should start from secure when
+            // reconnecting because regionserver may change its sasl config after restart.
+          }
+        }
+        this.in = new DataInputStream(new BufferedInputStream(inStream));
+        this.out = new DataOutputStream(new BufferedOutputStream(outStream));
+        // Now write out the connection header
+        writeConnectionHeader();
+        break;
+      }
+    } catch (Throwable t) {
+      closeSocket();
+      IOException e = ExceptionUtil.asInterrupt(t);
+      if (e == null) {
+        this.rpcClient.failedServers.addToFailedServers(remoteId.address);
+        if (t instanceof LinkageError) {
+          // probably the hbase hadoop version does not match the running hadoop version
+          e = new DoNotRetryIOException(t);
+        } else if (t instanceof IOException) {
+          e = (IOException) t;
+        } else {
+          e = new IOException("Could not set up IO Streams to " + remoteId.address, t);
+        }
+      }
+      throw e;
+    }
+
+    // start the receiver thread after the socket connection has been set up
+    thread = new Thread(this, threadName);
+    thread.setDaemon(true);
+    thread.start();
+  }
+
+  /**
+   * Write the RPC header: {@code <MAGIC WORD -- 'HBas'> <ONEBYTE_VERSION> <ONEBYTE_AUTH_TYPE>}
+   */
+  private void writeConnectionHeaderPreamble(OutputStream out) throws IOException {
+    out.write(connectionHeaderPreamble);
+    out.flush();
+  }
+
+  /**
+   * Write the connection header.
+   */
+  private void writeConnectionHeader() throws IOException {
+    this.out.write(connectionHeaderWithLength);
+    this.out.flush();
+  }
+
+  private void tracedWriteRequest(Call call) throws IOException {
+    try (TraceScope ignored = Trace.startSpan("RpcClientImpl.tracedWriteRequest", call.span)) {
+      writeRequest(call);
+    }
+  }
+
+  /**
+   * Initiates a call by sending the parameter to the remote server. Note: this is not called from
+   * the Connection thread, but by other threads.
+   * @see #readResponse()
+   */
+  private void writeRequest(Call call) throws IOException {
+    ByteBuffer cellBlock = this.rpcClient.cellBlockBuilder.buildCellBlock(this.codec,
+      this.compressor, call.cells);
+    CellBlockMeta cellBlockMeta;
+    if (cellBlock != null) {
+      cellBlockMeta = CellBlockMeta.newBuilder().setLength(cellBlock.limit()).build();
+    } else {
+      cellBlockMeta = null;
+    }
+    RequestHeader requestHeader = buildRequestHeader(call, cellBlockMeta);
+
+    setupIOstreams();
+
+    // Now we're going to write the call. We take the lock, then check that the connection
+    // is still valid, and, if so we do the write to the socket. If the write fails, we don't
+    // know where we stand, we have to close the connection.
+    if (Thread.interrupted()) {
+      throw new InterruptedIOException();
+    }
+
+    calls.put(call.id, call); // We put first as we don't want the connection to become idle.
+    // from here, we do not throw any exception to upper layer as the call has been tracked in the
+    // pending calls map.
+    try {
+      call.callStats.setRequestSizeBytes(write(this.out, requestHeader, call.param, cellBlock));
+    } catch (IOException e) {
+      closeConn(e);
+      return;
+    }
+    notifyAll();
+  }
+
+  /*
+   * Receive a response. Because only one receiver, so no synchronization on in.
+   */
+  private void readResponse() {
+    Call call = null;
+    boolean expectedCall = false;
+    try {
+      // See HBaseServer.Call.setResponse for where we write out the response.
+      // Total size of the response. Unused. But have to read it in anyways.
+      int totalSize = in.readInt();
+
+      // Read the header
+      ResponseHeader responseHeader = ResponseHeader.parseDelimitedFrom(in);
+      int id = responseHeader.getCallId();
+      call = calls.remove(id); // call.done have to be set before leaving this method
+      expectedCall = (call != null && !call.isDone());
+      if (!expectedCall) {
+        // So we got a response for which we have no corresponding 'call' here on the client-side.
+        // We probably timed out waiting, cleaned up all references, and now the server decides
+        // to return a response. There is nothing we can do w/ the response at this stage. Clean
+        // out the wire of the response so its out of the way and we can get other responses on
+        // this connection.
+        int readSoFar = getTotalSizeWhenWrittenDelimited(responseHeader);
+        int whatIsLeftToRead = totalSize - readSoFar;
+        IOUtils.skipFully(in, whatIsLeftToRead);
+        if (call != null) {
+          call.callStats.setResponseSizeBytes(totalSize);
+          call.callStats
+              .setCallTimeMs(EnvironmentEdgeManager.currentTime() - call.callStats.getStartTime());
+        }
+        return;
+      }
+      if (responseHeader.hasException()) {
+        ExceptionResponse exceptionResponse = responseHeader.getException();
+        RemoteException re = createRemoteException(exceptionResponse);
+        call.setException(re);
+        call.callStats.setResponseSizeBytes(totalSize);
+        call.callStats
+            .setCallTimeMs(EnvironmentEdgeManager.currentTime() - call.callStats.getStartTime());
+        if (isFatalConnectionException(exceptionResponse)) {
+          synchronized (this) {
+            closeConn(re);
+          }
+        }
+      } else {
+        Message value = null;
+        if (call.responseDefaultType != null) {
+          Builder builder = call.responseDefaultType.newBuilderForType();
+          ProtobufUtil.mergeDelimitedFrom(builder, in);
+          value = builder.build();
+        }
+        CellScanner cellBlockScanner = null;
+        if (responseHeader.hasCellBlockMeta()) {
+          int size = responseHeader.getCellBlockMeta().getLength();
+          byte[] cellBlock = new byte[size];
+          IOUtils.readFully(this.in, cellBlock, 0, cellBlock.length);
+          cellBlockScanner = this.rpcClient.cellBlockBuilder.createCellScanner(this.codec,
+            this.compressor, cellBlock);
+        }
+        call.setResponse(value, cellBlockScanner);
+        call.callStats.setResponseSizeBytes(totalSize);
+        call.callStats
+            .setCallTimeMs(EnvironmentEdgeManager.currentTime() - call.callStats.getStartTime());
+      }
+    } catch (IOException e) {
+      if (expectedCall) {
+        call.setException(e);
+      }
+      if (e instanceof SocketTimeoutException) {
+        // Clean up open calls but don't treat this as a fatal condition,
+        // since we expect certain responses to not make it by the specified
+        // {@link ConnectionId#rpcTimeout}.
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("ignored", e);
+        }
+      } else {
+        synchronized (this) {
+          closeConn(e);
+        }
+      }
+    }
+  }
+
+  @Override
+  protected synchronized void callTimeout(Call call) {
+    // call sender
+    calls.remove(call.id);
+  }
+
+  // just close socket input and output.
+  private void closeSocket() {
+    IOUtils.closeStream(out);
+    IOUtils.closeStream(in);
+    IOUtils.closeSocket(socket);
+    out = null;
+    in = null;
+    socket = null;
+  }
+
+  // close socket, reader, and clean up all pending calls.
+  private void closeConn(IOException e) {
+    if (thread == null) {
+      return;
+    }
+    thread.interrupt();
+    thread = null;
+    closeSocket();
+    if (callSender != null) {
+      callSender.cleanup(e);
+    }
+    for (Call call : calls.values()) {
+      call.setException(e);
+    }
+    calls.clear();
+  }
+
+  // release all resources, the connection will not be used any more.
+  @Override
+  public synchronized void shutdown() {
+    closed = true;
+    if (callSender != null) {
+      callSender.interrupt();
+    }
+    closeConn(new IOException("connection to " + remoteId.address + " closed"));
+  }
+
+  @Override
+  public synchronized void sendRequest(final Call call, HBaseRpcController pcrc)
+      throws IOException {
+    pcrc.notifyOnCancel(new RpcCallback<Object>() {
+
+      @Override
+      public void run(Object parameter) {
+        setCancelled(call);
+        synchronized (BlockingRpcConnection.this) {
+          if (callSender != null) {
+            callSender.remove(call);
+          } else {
+            calls.remove(call.id);
+          }
+        }
+      }
+    }, new CancellationCallback() {
+
+      @Override
+      public void run(boolean cancelled) throws IOException {
+        if (cancelled) {
+          setCancelled(call);
+          return;
+        }
+        scheduleTimeoutTask(call);
+        if (callSender != null) {
+          callSender.sendCall(call);
+        } else {
+          tracedWriteRequest(call);
+        }
+      }
+    });
+  }
+
+  @Override
+  public synchronized boolean isActive() {
+    return thread != null;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BufferCallBeforeInitHandler.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BufferCallBeforeInitHandler.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BufferCallBeforeInitHandler.java
new file mode 100644
index 0000000..573ddd5
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BufferCallBeforeInitHandler.java
@@ -0,0 +1,103 @@
+/**
+ * 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.ipc;
+
+import io.netty.channel.ChannelDuplexHandler;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelPromise;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * We will expose the connection to upper layer before initialized, so we need to buffer the calls
+ * passed in and write them out once the connection is established.
+ */
+@InterfaceAudience.Private
+class BufferCallBeforeInitHandler extends ChannelDuplexHandler {
+
+  private enum BufferCallAction {
+    FLUSH, FAIL
+  }
+
+  public static final class BufferCallEvent {
+
+    public final BufferCallAction action;
+
+    public final IOException error;
+
+    private BufferCallEvent(BufferCallBeforeInitHandler.BufferCallAction action,
+        IOException error) {
+      this.action = action;
+      this.error = error;
+    }
+
+    public static BufferCallBeforeInitHandler.BufferCallEvent success() {
+      return SUCCESS_EVENT;
+    }
+
+    public static BufferCallBeforeInitHandler.BufferCallEvent fail(IOException error) {
+      return new BufferCallEvent(BufferCallAction.FAIL, error);
+    }
+  }
+
+  private static final BufferCallEvent SUCCESS_EVENT = new BufferCallEvent(BufferCallAction.FLUSH,
+      null);
+
+  private final Map<Integer, Call> id2Call = new HashMap<Integer, Call>();
+
+  @Override
+  public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) {
+    if (msg instanceof Call) {
+      Call call = (Call) msg;
+      id2Call.put(call.id, call);
+    } else {
+      ctx.write(msg, promise);
+    }
+  }
+
+  @Override
+  public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exception {
+    if (evt instanceof BufferCallEvent) {
+      BufferCallEvent bcEvt = (BufferCallBeforeInitHandler.BufferCallEvent) evt;
+      switch (bcEvt.action) {
+        case FLUSH:
+          for (Call call : id2Call.values()) {
+            ctx.write(call);
+          }
+          break;
+        case FAIL:
+          for (Call call : id2Call.values()) {
+            call.setException(bcEvt.error);
+          }
+          break;
+      }
+      ctx.flush();
+      ctx.pipeline().remove(this);
+    } else if (evt instanceof CallEvent) {
+      // just remove the call for now until we add other call event other than timeout and cancel.
+      id2Call.remove(((CallEvent) evt).call.id);
+    } else {
+      ctx.fireUserEventTriggered(evt);
+    }
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/Call.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/Call.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/Call.java
index 73bc0e2..a6203d5 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/Call.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/Call.java
@@ -19,6 +19,9 @@ package org.apache.hadoop.hbase.ipc;
 
 import com.google.protobuf.Descriptors;
 import com.google.protobuf.Message;
+import com.google.protobuf.RpcCallback;
+
+import io.netty.util.Timeout;
 
 import java.io.IOException;
 
@@ -27,29 +30,39 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.MetricsConnection;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.htrace.Span;
+import org.apache.htrace.Trace;
 
 /** A call waiting for a value. */
 @InterfaceAudience.Private
-public class Call {
-  final int id;                                 // call id
-  final Message param;                          // rpc request method param object
+class Call {
+  final int id; // call id
+  final Message param; // rpc request method param object
   /**
-   * Optionally has cells when making call.  Optionally has cells set on response.  Used
-   * passing cells to the rpc and receiving the response.
+   * Optionally has cells when making call. Optionally has cells set on response. Used passing cells
+   * to the rpc and receiving the response.
    */
   CellScanner cells;
-  Message response;                             // value, null if error
-  // The return type.  Used to create shell into which we deserialize the response if any.
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "IS2_INCONSISTENT_SYNC",
+      justification = "Direct access is only allowed after done")
+  Message response; // value, null if error
+  // The return type. Used to create shell into which we deserialize the response if any.
   Message responseDefaultType;
-  IOException error;                            // exception, null if value
-  volatile boolean done;                                 // true when call is done
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "IS2_INCONSISTENT_SYNC",
+    justification = "Direct access is only allowed after done")
+  IOException error; // exception, null if value
+  private boolean done; // true when call is done
   final Descriptors.MethodDescriptor md;
   final int timeout; // timeout in millisecond for this call; 0 means infinite.
+  final int priority;
   final MetricsConnection.CallStats callStats;
+  final RpcCallback<Call> callback;
+  final Span span;
+  Timeout timeoutTask;
 
   protected Call(int id, final Descriptors.MethodDescriptor md, Message param,
-      final CellScanner cells, final Message responseDefaultType, int timeout,
-      MetricsConnection.CallStats callStats) {
+      final CellScanner cells, final Message responseDefaultType, int timeout, int priority,
+      RpcCallback<Call> callback, MetricsConnection.CallStats callStats) {
     this.param = param;
     this.md = md;
     this.cells = cells;
@@ -58,73 +71,74 @@ public class Call {
     this.responseDefaultType = responseDefaultType;
     this.id = id;
     this.timeout = timeout;
+    this.priority = priority;
+    this.callback = callback;
+    this.span = Trace.currentSpan();
+  }
+
+  @Override
+  public String toString() {
+    return "callId: " + this.id + " methodName: " + this.md.getName() + " param {"
+        + (this.param != null ? ProtobufUtil.getShortTextFormat(this.param) : "") + "}";
   }
 
   /**
-   * Check if the call did timeout. Set an exception (includes a notify) if it's the case.
-   * @return true if the call is on timeout, false otherwise.
+   * called from timeoutTask, prevent self cancel
    */
-  public boolean checkAndSetTimeout() {
-    if (timeout == 0){
-      return false;
-    }
-
-    long waitTime = EnvironmentEdgeManager.currentTime() - getStartTime();
-    if (waitTime >= timeout) {
-      IOException ie = new CallTimeoutException("Call id=" + id +
-          ", waitTime=" + waitTime + ", operationTimeout=" + timeout + " expired.");
-      setException(ie); // includes a notify
-      return true;
-    } else {
-      return false;
+  public void setTimeout(IOException error) {
+    synchronized (this) {
+      if (done) {
+        return;
+      }
+      this.done = true;
+      this.error = error;
     }
+    callback.run(this);
   }
 
-  public int remainingTime() {
-    if (timeout == 0) {
-      return Integer.MAX_VALUE;
+  private void callComplete() {
+    if (timeoutTask != null) {
+      timeoutTask.cancel();
     }
-
-    int remaining = timeout - (int) (EnvironmentEdgeManager.currentTime() - getStartTime());
-    return remaining > 0 ? remaining : 0;
-  }
-
-  @Override
-  public String toString() {
-    return "callId: " + this.id + " methodName: " + this.md.getName() + " param {" +
-      (this.param != null? ProtobufUtil.getShortTextFormat(this.param): "") + "}";
+    callback.run(this);
   }
 
-  /** Indicate when the call is complete and the
-   * value or error are available.  Notifies by default.  */
-  protected synchronized void callComplete() {
-    this.done = true;
-    notify();                                 // notify caller
-  }
-
-  /** Set the exception when there is an error.
-   * Notify the caller the call is done.
-   *
+  /**
+   * Set the exception when there is an error. Notify the caller the call is done.
    * @param error exception thrown by the call; either local or remote
    */
   public void setException(IOException error) {
-    this.error = error;
+    synchronized (this) {
+      if (done) {
+        return;
+      }
+      this.done = true;
+      this.error = error;
+    }
     callComplete();
   }
 
   /**
-   * Set the return value when there is no error.
-   * Notify the caller the call is done.
-   *
+   * Set the return value when there is no error. Notify the caller the call is done.
    * @param response return value of the call.
    * @param cells Can be null
    */
   public void setResponse(Message response, final CellScanner cells) {
-    this.response = response;
-    this.cells = cells;
+    synchronized (this) {
+      if (done) {
+        return;
+      }
+      this.done = true;
+      this.response = response;
+      this.cells = cells;
+    }
     callComplete();
   }
 
+  public synchronized boolean isDone() {
+    return done;
+  }
+
   public long getStartTime() {
     return this.callStats.getStartTime();
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CallCancelledException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CallCancelledException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CallCancelledException.java
new file mode 100644
index 0000000..a6777c0
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CallCancelledException.java
@@ -0,0 +1,37 @@
+/**
+ * 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.ipc;
+
+import org.apache.hadoop.hbase.HBaseIOException;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * Client side call cancelled.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class CallCancelledException extends HBaseIOException {
+
+  private static final long serialVersionUID = 309775809470318208L;
+
+  public CallCancelledException(String message) {
+    super(message);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CallEvent.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CallEvent.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CallEvent.java
new file mode 100644
index 0000000..1c2ea32
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CallEvent.java
@@ -0,0 +1,40 @@
+/**
+ * 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.ipc;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * Used to tell netty handler the call is cancelled, timeout...
+ */
+@InterfaceAudience.Private
+class CallEvent {
+
+  public enum Type {
+    TIMEOUT, CANCELLED
+  }
+
+  final Type type;
+
+  final Call call;
+
+  CallEvent(Type type, Call call) {
+    this.type = type;
+    this.call = call;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CallTimeoutException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CallTimeoutException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CallTimeoutException.java
index 1e31f72..db8c34a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CallTimeoutException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CallTimeoutException.java
@@ -17,8 +17,7 @@
  */
 package org.apache.hadoop.hbase.ipc;
 
-import java.io.IOException;
-
+import org.apache.hadoop.hbase.HBaseIOException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 
@@ -28,7 +27,8 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
 @SuppressWarnings("serial")
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
-public class CallTimeoutException extends IOException {
+public class CallTimeoutException extends HBaseIOException {
+
   public CallTimeoutException(final String msg) {
     super(msg);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CellBlockBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CellBlockBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CellBlockBuilder.java
index 072a490..fb2cafa 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CellBlockBuilder.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CellBlockBuilder.java
@@ -17,6 +17,10 @@
  */
 package org.apache.hadoop.hbase.ipc;
 
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.buffer.ByteBufOutputStream;
+
 import java.io.IOException;
 import java.io.OutputStream;
 import java.nio.BufferOverflowException;
@@ -46,12 +50,13 @@ import org.apache.hadoop.io.compress.Decompressor;
  * Helper class for building cell block.
  */
 @InterfaceAudience.Private
-public class CellBlockBuilder {
+class CellBlockBuilder {
 
   // LOG is being used in TestCellBlockBuilder
   static final Log LOG = LogFactory.getLog(CellBlockBuilder.class);
 
   private final Configuration conf;
+
   /**
    * How much we think the decompressor will expand the original compressed content.
    */
@@ -59,7 +64,7 @@ public class CellBlockBuilder {
 
   private final int cellBlockBuildingInitialBufferSize;
 
-  public CellBlockBuilder(final Configuration conf) {
+  public CellBlockBuilder(Configuration conf) {
     this.conf = conf;
     this.cellBlockDecompressionMultiplier = conf
         .getInt("hbase.ipc.cellblock.decompression.buffersize.multiplier", 3);
@@ -70,44 +75,104 @@ public class CellBlockBuilder {
         .align(conf.getInt("hbase.ipc.cellblock.building.initial.buffersize", 16 * 1024));
   }
 
+  private interface OutputStreamSupplier {
+
+    OutputStream get(int expectedSize);
+
+    int size();
+  }
+
+  private static final class ByteBufferOutputStreamSupplier implements OutputStreamSupplier {
+
+    private ByteBufferOutputStream baos;
+
+    @Override
+    public OutputStream get(int expectedSize) {
+      baos = new ByteBufferOutputStream(expectedSize);
+      return baos;
+    }
+
+    @Override
+    public int size() {
+      return baos.size();
+    }
+  }
+
   /**
    * Puts CellScanner Cells into a cell block using passed in <code>codec</code> and/or
    * <code>compressor</code>.
-   * @param codec to use for encoding
-   * @param compressor to use for encoding
-   * @param cellScanner to encode
+   * @param codec
+   * @param compressor
+   * @param cellScanner
    * @return Null or byte buffer filled with a cellblock filled with passed-in Cells encoded using
    *         passed in <code>codec</code> and/or <code>compressor</code>; the returned buffer has
    *         been flipped and is ready for reading. Use limit to find total size.
-   * @throws IOException if encoding the cells fail
+   * @throws IOException
    */
   public ByteBuffer buildCellBlock(final Codec codec, final CompressionCodec compressor,
       final CellScanner cellScanner) throws IOException {
-    if (cellScanner == null) {
+    ByteBufferOutputStreamSupplier supplier = new ByteBufferOutputStreamSupplier();
+    if (buildCellBlock(codec, compressor, cellScanner, supplier)) {
+      ByteBuffer bb = supplier.baos.getByteBuffer();
+      // If no cells, don't mess around. Just return null (could be a bunch of existence checking
+      // gets or something -- stuff that does not return a cell).
+      return bb.hasRemaining() ? bb : null;
+    } else {
       return null;
     }
+  }
+
+  private static final class ByteBufOutputStreamSupplier implements OutputStreamSupplier {
+
+    private final ByteBufAllocator alloc;
+
+    private ByteBuf buf;
+
+    public ByteBufOutputStreamSupplier(ByteBufAllocator alloc) {
+      this.alloc = alloc;
+    }
+
+    @Override
+    public OutputStream get(int expectedSize) {
+      buf = alloc.buffer(expectedSize);
+      return new ByteBufOutputStream(buf);
+    }
+
+    @Override
+    public int size() {
+      return buf.writerIndex();
+    }
+  }
+
+  public ByteBuf buildCellBlock(Codec codec, CompressionCodec compressor, CellScanner cellScanner,
+      ByteBufAllocator alloc) throws IOException {
+    ByteBufOutputStreamSupplier supplier = new ByteBufOutputStreamSupplier(alloc);
+    if (buildCellBlock(codec, compressor, cellScanner, supplier)) {
+      return supplier.buf;
+    } else {
+      return null;
+    }
+  }
+
+  private boolean buildCellBlock(final Codec codec, final CompressionCodec compressor,
+      final CellScanner cellScanner, OutputStreamSupplier supplier) throws IOException {
+    if (cellScanner == null) {
+      return false;
+    }
     if (codec == null) {
       throw new CellScannerButNoCodecException();
     }
-    int bufferSize = this.cellBlockBuildingInitialBufferSize;
-    ByteBufferOutputStream baos = new ByteBufferOutputStream(bufferSize);
-    encodeCellsTo(baos, cellScanner, codec, compressor);
-    if (LOG.isTraceEnabled()) {
-      if (bufferSize < baos.size()) {
-        LOG.trace("Buffer grew from initial bufferSize=" + bufferSize + " to " + baos.size()
-            + "; up hbase.ipc.cellblock.building.initial.buffersize?");
-      }
+    int bufferSize = cellBlockBuildingInitialBufferSize;
+    encodeCellsTo(supplier.get(bufferSize), cellScanner, codec, compressor);
+    if (LOG.isTraceEnabled() && bufferSize < supplier.size()) {
+      LOG.trace("Buffer grew from initial bufferSize=" + bufferSize + " to " + supplier.size()
+          + "; up hbase.ipc.cellblock.building.initial.buffersize?");
     }
-    ByteBuffer bb = baos.getByteBuffer();
-    // If no cells, don't mess around. Just return null (could be a bunch of existence checking
-    // gets or something -- stuff that does not return a cell).
-    if (!bb.hasRemaining()) return null;
-    return bb;
+    return true;
   }
 
-  private void encodeCellsTo(ByteBufferOutputStream bbos, CellScanner cellScanner, Codec codec,
+  private void encodeCellsTo(OutputStream os, CellScanner cellScanner, Codec codec,
       CompressionCodec compressor) throws IOException {
-    OutputStream os = bbos;
     Compressor poolCompressor = null;
     try {
       if (compressor != null) {
@@ -122,7 +187,7 @@ public class CellBlockBuilder {
         encoder.write(cellScanner.current());
       }
       encoder.flush();
-    } catch (BufferOverflowException e) {
+    } catch (BufferOverflowException | IndexOutOfBoundsException e) {
       throw new DoNotRetryIOException(e);
     } finally {
       os.close();

http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/ConnectionId.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/ConnectionId.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/ConnectionId.java
index 08f8171..1b837d8 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/ConnectionId.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/ConnectionId.java
@@ -27,7 +27,7 @@ import org.apache.hadoop.hbase.security.User;
  * to servers are uniquely identified by &lt;remoteAddress, ticket, serviceName&gt;
  */
 @InterfaceAudience.Private
-public class ConnectionId {
+class ConnectionId {
   private static final int PRIME = 16777619;
   final User ticket;
   final String serviceName;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/DefaultNettyEventLoopConfig.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/DefaultNettyEventLoopConfig.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/DefaultNettyEventLoopConfig.java
new file mode 100644
index 0000000..c7c0f32
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/DefaultNettyEventLoopConfig.java
@@ -0,0 +1,37 @@
+/**
+ * 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.ipc;
+
+import io.netty.channel.Channel;
+import io.netty.channel.EventLoopGroup;
+import io.netty.channel.nio.NioEventLoopGroup;
+import io.netty.channel.socket.nio.NioSocketChannel;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.util.Pair;
+
+/**
+ * The default netty event loop config
+ */
+@InterfaceAudience.Private
+class DefaultNettyEventLoopConfig {
+
+  public static final Pair<EventLoopGroup, Class<? extends Channel>> GROUP_AND_CHANNEL_CLASS = Pair
+      .<EventLoopGroup, Class<? extends Channel>> newPair(new NioEventLoopGroup(),
+        NioSocketChannel.class);
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/FallbackDisallowedException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/FallbackDisallowedException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/FallbackDisallowedException.java
new file mode 100644
index 0000000..721148b
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/FallbackDisallowedException.java
@@ -0,0 +1,38 @@
+/**
+ * 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.ipc;
+
+import org.apache.hadoop.hbase.HBaseIOException;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * Indicate that the rpc server tells client to fallback to simple auth but client is disabled to do
+ * so.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class FallbackDisallowedException extends HBaseIOException {
+
+  private static final long serialVersionUID = -6942845066279358253L;
+
+  public FallbackDisallowedException() {
+    super("Server asks us to fall back to SIMPLE auth, "
+        + "but this client is configured to only allow secure connections.");
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/c04b3891/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IOExceptionConverter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IOExceptionConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IOExceptionConverter.java
deleted file mode 100644
index 09dda09..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IOExceptionConverter.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/**
- * 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.ipc;
-
-import java.io.IOException;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-
-/**
- * Converts exceptions to other exceptions
- */
-@InterfaceAudience.Private
-public interface IOExceptionConverter {
-  /**
-   * Converts given IOException
-   * @param e exception to convert
-   * @return converted IOException
-   */
-  IOException convert(IOException e);
-}