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 2019/01/04 06:40:27 UTC

[01/15] hbase git commit: HBASE-21547 Precommit uses master flaky list for other branches [Forced Update!]

Repository: hbase
Updated Branches:
  refs/heads/HBASE-21512 44462a48e -> 04e6909ad (forced update)


HBASE-21547 Precommit uses master flaky list for other branches

Signed-off-by: Sean Busbey <bu...@apache.org>


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

Branch: refs/heads/HBASE-21512
Commit: f0b50a8f9b6db266c1e0f307fda6a4ad7b852135
Parents: ec948f5
Author: Peter Somogyi <ps...@apache.org>
Authored: Fri Dec 28 15:16:48 2018 +0100
Committer: Peter Somogyi <ps...@apache.org>
Committed: Wed Jan 2 08:26:57 2019 +0100

----------------------------------------------------------------------
 dev-support/hbase-personality.sh | 14 ++++++++++++++
 1 file changed, 14 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/f0b50a8f/dev-support/hbase-personality.sh
----------------------------------------------------------------------
diff --git a/dev-support/hbase-personality.sh b/dev-support/hbase-personality.sh
index 3856a29..8dd24be 100755
--- a/dev-support/hbase-personality.sh
+++ b/dev-support/hbase-personality.sh
@@ -272,6 +272,20 @@ function get_include_exclude_tests_arg
         yetus_error "Wget error $? in fetching includes file from url" \
              "${INCLUDE_TESTS_URL}. Ignoring and proceeding."
       fi
+  else
+    # Use branch specific exclude list when EXCLUDE_TESTS_URL and INCLUDE_TESTS_URL are empty
+    FLAKY_URL="https://builds.apache.org/job/HBase-Find-Flaky-Tests/job/${PATCH_BRANCH}/lastSuccessfulBuild/artifact/excludes/"
+    if wget "${FLAKY_URL}" -O "excludes"; then
+      excludes=$(cat excludes)
+        yetus_debug "excludes=${excludes}"
+        if [[ -n "${excludes}" ]]; then
+          eval "${__resultvar}='-Dtest.exclude.pattern=${excludes}'"
+        fi
+        rm excludes
+      else
+        yetus_error "Wget error $? in fetching excludes file from url" \
+             "${FLAKY_URL}. Ignoring and proceeding."
+      fi
   fi
 }
 


[05/15] hbase git commit: HBASE-21652 Refactor ThriftServer making thrift2 server inherited from thrift1 server

Posted by zh...@apache.org.
HBASE-21652 Refactor ThriftServer making thrift2 server inherited from thrift1 server


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

Branch: refs/heads/HBASE-21512
Commit: e4b6b4afb933a961f543537875f87a2dc62d3757
Parents: f0b50a8
Author: Allan Yang <al...@apache.org>
Authored: Wed Jan 2 16:13:17 2019 +0800
Committer: Allan Yang <al...@apache.org>
Committed: Wed Jan 2 16:13:57 2019 +0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/thrift/Constants.java   |  151 ++
 .../hbase/thrift/HBaseServiceHandler.java       |   90 +
 .../hbase/thrift/HbaseHandlerMetricsProxy.java  |   20 +-
 .../apache/hadoop/hbase/thrift/ImplType.java    |  143 ++
 .../hadoop/hbase/thrift/IncrementCoalescer.java |    6 +-
 .../hbase/thrift/ThriftHBaseServiceHandler.java | 1347 ++++++++++++
 .../hadoop/hbase/thrift/ThriftHttpServlet.java  |   12 +-
 .../hadoop/hbase/thrift/ThriftServer.java       |  698 +++++-
 .../hadoop/hbase/thrift/ThriftServerRunner.java | 2031 ------------------
 .../thrift2/ThriftHBaseServiceHandler.java      |   69 +-
 .../hadoop/hbase/thrift2/ThriftServer.java      |  594 +----
 .../resources/hbase-webapps/thrift/thrift.jsp   |    2 +-
 .../hbase/thrift/TestThriftHttpServer.java      |   28 +-
 .../hadoop/hbase/thrift/TestThriftServer.java   |   58 +-
 .../hbase/thrift/TestThriftServerCmdLine.java   |   48 +-
 .../thrift/TestThriftSpnegoHttpServer.java      |   21 +-
 .../hbase/thrift2/TestThrift2HttpServer.java    |   90 +
 .../hbase/thrift2/TestThrift2ServerCmdLine.java |   99 +
 .../thrift2/TestThriftHBaseServiceHandler.java  |   15 +-
 19 files changed, 2711 insertions(+), 2811 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b6b4af/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/Constants.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/Constants.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/Constants.java
new file mode 100644
index 0000000..8e3d004
--- /dev/null
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/Constants.java
@@ -0,0 +1,151 @@
+/*
+ * 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.thrift;
+
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Thrift related constants
+ */
+@InterfaceAudience.Private
+public final class Constants {
+  private Constants(){}
+
+  public static final int DEFAULT_HTTP_MAX_HEADER_SIZE = 64 * 1024; // 64k
+
+  public static final String SERVER_TYPE_CONF_KEY =
+      "hbase.regionserver.thrift.server.type";
+
+  public static final String COMPACT_CONF_KEY = "hbase.regionserver.thrift.compact";
+  public static final boolean COMPACT_CONF_DEFAULT = false;
+
+  public static final String FRAMED_CONF_KEY = "hbase.regionserver.thrift.framed";
+  public static final boolean FRAMED_CONF_DEFAULT = false;
+
+  public static final String MAX_FRAME_SIZE_CONF_KEY =
+      "hbase.regionserver.thrift.framed.max_frame_size_in_mb";
+  public static final int MAX_FRAME_SIZE_CONF_DEFAULT = 2;
+
+  public static final String COALESCE_INC_KEY = "hbase.regionserver.thrift.coalesceIncrement";
+  public static final String USE_HTTP_CONF_KEY = "hbase.regionserver.thrift.http";
+
+  public static final String HTTP_MIN_THREADS_KEY = "hbase.thrift.http_threads.min";
+  public static final int HTTP_MIN_THREADS_KEY_DEFAULT = 2;
+
+  public static final String HTTP_MAX_THREADS_KEY = "hbase.thrift.http_threads.max";
+  public static final int HTTP_MAX_THREADS_KEY_DEFAULT = 100;
+
+  // ssl related configs
+  public static final String THRIFT_SSL_ENABLED_KEY = "hbase.thrift.ssl.enabled";
+  public static final String THRIFT_SSL_KEYSTORE_STORE_KEY = "hbase.thrift.ssl.keystore.store";
+  public static final String THRIFT_SSL_KEYSTORE_PASSWORD_KEY =
+      "hbase.thrift.ssl.keystore.password";
+  public static final String THRIFT_SSL_KEYSTORE_KEYPASSWORD_KEY
+      = "hbase.thrift.ssl.keystore.keypassword";
+  public static final String THRIFT_SSL_EXCLUDE_CIPHER_SUITES_KEY =
+      "hbase.thrift.ssl.exclude.cipher.suites";
+  public static final String THRIFT_SSL_INCLUDE_CIPHER_SUITES_KEY =
+      "hbase.thrift.ssl.include.cipher.suites";
+  public static final String THRIFT_SSL_EXCLUDE_PROTOCOLS_KEY =
+      "hbase.thrift.ssl.exclude.protocols";
+  public static final String THRIFT_SSL_INCLUDE_PROTOCOLS_KEY =
+      "hbase.thrift.ssl.include.protocols";
+
+
+  public static final String THRIFT_SUPPORT_PROXYUSER_KEY = "hbase.thrift.support.proxyuser";
+
+  //kerberos related configs
+  public static final String THRIFT_DNS_INTERFACE_KEY = "hbase.thrift.dns.interface";
+  public static final String THRIFT_DNS_NAMESERVER_KEY = "hbase.thrift.dns.nameserver";
+  public static final String THRIFT_KERBEROS_PRINCIPAL_KEY = "hbase.thrift.kerberos.principal";
+  public static final String THRIFT_KEYTAB_FILE_KEY = "hbase.thrift.keytab.file";
+  public static final String THRIFT_SPNEGO_PRINCIPAL_KEY = "hbase.thrift.spnego.principal";
+  public static final String THRIFT_SPNEGO_KEYTAB_FILE_KEY = "hbase.thrift.spnego.keytab.file";
+
+  /**
+   * Amount of time in milliseconds before a server thread will timeout
+   * waiting for client to send data on a connected socket. Currently,
+   * applies only to TBoundedThreadPoolServer
+   */
+  public static final String THRIFT_SERVER_SOCKET_READ_TIMEOUT_KEY =
+      "hbase.thrift.server.socket.read.timeout";
+  public static final int THRIFT_SERVER_SOCKET_READ_TIMEOUT_DEFAULT = 60000;
+
+
+  /**
+   * Thrift quality of protection configuration key. Valid values can be:
+   * auth-conf: authentication, integrity and confidentiality checking
+   * auth-int: authentication and integrity checking
+   * auth: authentication only
+   *
+   * This is used to authenticate the callers and support impersonation.
+   * The thrift server and the HBase cluster must run in secure mode.
+   */
+  public static final String THRIFT_QOP_KEY = "hbase.thrift.security.qop";
+
+  public static final String BACKLOG_CONF_KEY = "hbase.regionserver.thrift.backlog";
+  public static final int BACKLOG_CONF_DEAFULT = 0;
+
+  public static final String BIND_CONF_KEY = "hbase.regionserver.thrift.ipaddress";
+  public static final String DEFAULT_BIND_ADDR = "0.0.0.0";
+
+  public static final String PORT_CONF_KEY = "hbase.regionserver.thrift.port";
+  public static final int DEFAULT_LISTEN_PORT = 9090;
+
+  public static final String THRIFT_HTTP_ALLOW_OPTIONS_METHOD =
+      "hbase.thrift.http.allow.options.method";
+  public static final boolean THRIFT_HTTP_ALLOW_OPTIONS_METHOD_DEFAULT = false;
+
+  public static final String THRIFT_INFO_SERVER_PORT = "hbase.thrift.info.port";
+  public static final int THRIFT_INFO_SERVER_PORT_DEFAULT = 9095;
+
+  public static final String THRIFT_INFO_SERVER_BINDING_ADDRESS = "hbase.thrift.info.bindAddress";
+  public static final String THRIFT_INFO_SERVER_BINDING_ADDRESS_DEFAULT = "0.0.0.0";
+
+  public static final String THRIFT_QUEUE_SIZE = "hbase.thrift.queue.size";
+  public static final int THRIFT_QUEUE_SIZE_DEFAULT = Integer.MAX_VALUE;
+
+  public static final String THRIFT_SELECTOR_NUM = "hbase.thrift.selector.num";
+
+  public static final String THRIFT_FILTERS = "hbase.thrift.filters";
+
+  // Command line options
+
+  public static final String READ_TIMEOUT_OPTION = "readTimeout";
+  public static final String MIN_WORKERS_OPTION = "minWorkers";
+  public static final String MAX_WORKERS_OPTION = "workers";
+  public static final String MAX_QUEUE_SIZE_OPTION = "queue";
+  public static final String SELECTOR_NUM_OPTION = "selectors";
+  public static final String KEEP_ALIVE_SEC_OPTION = "keepAliveSec";
+  public static final String BIND_OPTION = "bind";
+  public static final String COMPACT_OPTION = "compact";
+  public static final String FRAMED_OPTION = "framed";
+  public static final String PORT_OPTION = "port";
+  public static final String INFOPORT_OPTION = "infoport";
+
+  //for thrift2 server
+  public static final String READONLY_OPTION ="readonly";
+
+  public static final String THRIFT_READONLY_ENABLED = "hbase.thrift.readonly";
+  public static final boolean THRIFT_READONLY_ENABLED_DEFAULT = false;
+
+
+
+
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b6b4af/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/HBaseServiceHandler.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/HBaseServiceHandler.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/HBaseServiceHandler.java
new file mode 100644
index 0000000..7990871
--- /dev/null
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/HBaseServiceHandler.java
@@ -0,0 +1,90 @@
+/**
+ *
+ * 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.thrift;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.security.UserProvider;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.ConnectionCache;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * abstract class for HBase handler
+ * providing a Connection cache and get table/admin method
+ */
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
+public abstract class HBaseServiceHandler {
+  public static final String CLEANUP_INTERVAL = "hbase.thrift.connection.cleanup-interval";
+  public static final String MAX_IDLETIME = "hbase.thrift.connection.max-idletime";
+
+  protected Configuration conf;
+
+  protected final ConnectionCache connectionCache;
+
+  public HBaseServiceHandler(final Configuration c,
+      final UserProvider userProvider) throws IOException {
+    this.conf = c;
+    int cleanInterval = conf.getInt(CLEANUP_INTERVAL, 10 * 1000);
+    int maxIdleTime = conf.getInt(MAX_IDLETIME, 10 * 60 * 1000);
+    connectionCache = new ConnectionCache(
+        conf, userProvider, cleanInterval, maxIdleTime);
+  }
+
+  protected ThriftMetrics metrics = null;
+
+  public void initMetrics(ThriftMetrics metrics) {
+    this.metrics = metrics;
+  }
+
+  public void setEffectiveUser(String effectiveUser) {
+    connectionCache.setEffectiveUser(effectiveUser);
+  }
+
+  /**
+   * Obtain HBaseAdmin. Creates the instance if it is not already created.
+   */
+  protected Admin getAdmin() throws IOException {
+    return connectionCache.getAdmin();
+  }
+
+  /**
+   * Creates and returns a Table instance from a given table name.
+   *
+   * @param tableName
+   *          name of table
+   * @return Table object
+   * @throws IOException if getting the table fails
+   */
+  protected Table getTable(final byte[] tableName) throws IOException {
+    String table = Bytes.toString(tableName);
+    return connectionCache.getTable(table);
+  }
+
+  protected Table getTable(final ByteBuffer tableName) throws IOException {
+    return getTable(Bytes.getBytes(tableName));
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b6b4af/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/HbaseHandlerMetricsProxy.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/HbaseHandlerMetricsProxy.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/HbaseHandlerMetricsProxy.java
index 5a6e436..1402f86 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/HbaseHandlerMetricsProxy.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/HbaseHandlerMetricsProxy.java
@@ -25,9 +25,8 @@ import java.lang.reflect.Proxy;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.thrift.generated.Hbase;
+import org.apache.hadoop.hbase.thrift2.generated.THBaseService;
 import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /**
  * Converts a Hbase.Iface using InvocationHandler so that it reports process
@@ -36,10 +35,7 @@ import org.slf4j.LoggerFactory;
 @InterfaceAudience.Private
 public final class HbaseHandlerMetricsProxy implements InvocationHandler {
 
-  private static final Logger LOG = LoggerFactory.getLogger(
-      HbaseHandlerMetricsProxy.class);
-
-  private final Hbase.Iface handler;
+  private final Object handler;
   private final ThriftMetrics metrics;
 
   public static Hbase.Iface newInstance(Hbase.Iface handler,
@@ -51,8 +47,18 @@ public final class HbaseHandlerMetricsProxy implements InvocationHandler {
         new HbaseHandlerMetricsProxy(handler, metrics, conf));
   }
 
+  // for thrift 2
+  public static THBaseService.Iface newInstance(THBaseService.Iface handler,
+      ThriftMetrics metrics,
+      Configuration conf) {
+    return (THBaseService.Iface) Proxy.newProxyInstance(
+        handler.getClass().getClassLoader(),
+        new Class[]{THBaseService.Iface.class},
+        new HbaseHandlerMetricsProxy(handler, metrics, conf));
+  }
+
   private HbaseHandlerMetricsProxy(
-      Hbase.Iface handler, ThriftMetrics metrics, Configuration conf) {
+      Object handler, ThriftMetrics metrics, Configuration conf) {
     this.handler = handler;
     this.metrics = metrics;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b6b4af/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ImplType.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ImplType.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ImplType.java
new file mode 100644
index 0000000..7108115
--- /dev/null
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ImplType.java
@@ -0,0 +1,143 @@
+/*
+ * 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.thrift;
+
+import static org.apache.hadoop.hbase.thrift.Constants.SERVER_TYPE_CONF_KEY;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.thrift.server.THsHaServer;
+import org.apache.thrift.server.TNonblockingServer;
+import org.apache.thrift.server.TServer;
+import org.apache.thrift.server.TThreadedSelectorServer;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
+import org.apache.hbase.thirdparty.org.apache.commons.cli.Option;
+import org.apache.hbase.thirdparty.org.apache.commons.cli.OptionGroup;
+
+/** An enum of server implementation selections */
+@InterfaceAudience.Private
+public enum ImplType {
+  HS_HA("hsha", true, THsHaServer.class, true),
+  NONBLOCKING("nonblocking", true, TNonblockingServer.class, true),
+  THREAD_POOL("threadpool", false, TBoundedThreadPoolServer.class, true),
+  THREADED_SELECTOR("threadedselector", true, TThreadedSelectorServer.class, true);
+
+  private static final Logger LOG = LoggerFactory.getLogger(ImplType.class);
+  public static final ImplType DEFAULT = THREAD_POOL;
+
+
+  final String option;
+  final boolean isAlwaysFramed;
+  final Class<? extends TServer> serverClass;
+  final boolean canSpecifyBindIP;
+
+  private ImplType(String option, boolean isAlwaysFramed,
+      Class<? extends TServer> serverClass, boolean canSpecifyBindIP) {
+    this.option = option;
+    this.isAlwaysFramed = isAlwaysFramed;
+    this.serverClass = serverClass;
+    this.canSpecifyBindIP = canSpecifyBindIP;
+  }
+
+  /**
+   * @return <code>-option</code>
+   */
+  @Override
+  public String toString() {
+    return "-" + option;
+  }
+
+  public String getOption() {
+    return option;
+  }
+
+  public boolean isAlwaysFramed() {
+    return isAlwaysFramed;
+  }
+
+  public String getDescription() {
+    StringBuilder sb = new StringBuilder("Use the " +
+        serverClass.getSimpleName());
+    if (isAlwaysFramed) {
+      sb.append(" This implies the framed transport.");
+    }
+    if (this == DEFAULT) {
+      sb.append("This is the default.");
+    }
+    return sb.toString();
+  }
+
+  static OptionGroup createOptionGroup() {
+    OptionGroup group = new OptionGroup();
+    for (ImplType t : values()) {
+      group.addOption(new Option(t.option, t.getDescription()));
+    }
+    return group;
+  }
+
+  public static ImplType getServerImpl(Configuration conf) {
+    String confType = conf.get(SERVER_TYPE_CONF_KEY, THREAD_POOL.option);
+    for (ImplType t : values()) {
+      if (confType.equals(t.option)) {
+        return t;
+      }
+    }
+    throw new AssertionError("Unknown server ImplType.option:" + confType);
+  }
+
+  static void setServerImpl(CommandLine cmd, Configuration conf) {
+    ImplType chosenType = null;
+    int numChosen = 0;
+    for (ImplType t : values()) {
+      if (cmd.hasOption(t.option)) {
+        chosenType = t;
+        ++numChosen;
+      }
+    }
+    if (numChosen < 1) {
+      LOG.info("Using default thrift server type");
+      chosenType = DEFAULT;
+    } else if (numChosen > 1) {
+      throw new AssertionError("Exactly one option out of " +
+          Arrays.toString(values()) + " has to be specified");
+    }
+    LOG.info("Using thrift server type " + chosenType.option);
+    conf.set(SERVER_TYPE_CONF_KEY, chosenType.option);
+  }
+
+  public String simpleClassName() {
+    return serverClass.getSimpleName();
+  }
+
+  public static List<String> serversThatCannotSpecifyBindIP() {
+    List<String> l = new ArrayList<>();
+    for (ImplType t : values()) {
+      if (!t.canSpecifyBindIP) {
+        l.add(t.simpleClassName());
+      }
+    }
+    return l;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b6b4af/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/IncrementCoalescer.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/IncrementCoalescer.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/IncrementCoalescer.java
index e36d639..971cd17 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/IncrementCoalescer.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/IncrementCoalescer.java
@@ -33,7 +33,6 @@ import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.LongAdder;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.thrift.ThriftServerRunner.HBaseHandler;
 import org.apache.hadoop.hbase.thrift.generated.TIncrement;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Threads;
@@ -180,7 +179,7 @@ public class IncrementCoalescer implements IncrementCoalescerMBean {
   private final ConcurrentMap<FullyQualifiedRow, Long> countersMap =
       new ConcurrentHashMap<>(100000, 0.75f, 1500);
   private final ThreadPoolExecutor pool;
-  private final HBaseHandler handler;
+  private final ThriftHBaseServiceHandler handler;
 
   private int maxQueueSize = 500000;
   private static final int CORE_POOL_SIZE = 1;
@@ -188,7 +187,7 @@ public class IncrementCoalescer implements IncrementCoalescerMBean {
   private static final Logger LOG = LoggerFactory.getLogger(FullyQualifiedRow.class);
 
   @SuppressWarnings("deprecation")
-  public IncrementCoalescer(HBaseHandler hand) {
+  public IncrementCoalescer(ThriftHBaseServiceHandler hand) {
     this.handler = hand;
     LinkedBlockingQueue<Runnable> queue = new LinkedBlockingQueue<>();
     pool =
@@ -230,6 +229,7 @@ public class IncrementCoalescer implements IncrementCoalescerMBean {
       inc.getAmmount());
   }
 
+  @SuppressWarnings("FutureReturnValueIgnored")
   private boolean internalQueueIncrement(byte[] tableName, byte[] rowKey, byte[] fam,
       byte[] qual, long ammount) throws TException {
     int countersMapSize = countersMap.size();

http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b6b4af/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftHBaseServiceHandler.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftHBaseServiceHandler.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftHBaseServiceHandler.java
new file mode 100644
index 0000000..34bf5e8
--- /dev/null
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftHBaseServiceHandler.java
@@ -0,0 +1,1347 @@
+/**
+ *
+ * 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.thrift;
+
+import static org.apache.hadoop.hbase.thrift.Constants.COALESCE_INC_KEY;
+import static org.apache.hadoop.hbase.util.Bytes.getBytes;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellBuilder;
+import org.apache.hadoop.hbase.CellBuilderFactory;
+import org.apache.hadoop.hbase.CellBuilderType;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.TableNotFoundException;
+import org.apache.hadoop.hbase.client.Append;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Durability;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Increment;
+import org.apache.hadoop.hbase.client.OperationWithAttributes;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionLocator;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.ParseFilter;
+import org.apache.hadoop.hbase.filter.PrefixFilter;
+import org.apache.hadoop.hbase.filter.WhileMatchFilter;
+import org.apache.hadoop.hbase.security.UserProvider;
+import org.apache.hadoop.hbase.thrift.generated.AlreadyExists;
+import org.apache.hadoop.hbase.thrift.generated.BatchMutation;
+import org.apache.hadoop.hbase.thrift.generated.ColumnDescriptor;
+import org.apache.hadoop.hbase.thrift.generated.Hbase;
+import org.apache.hadoop.hbase.thrift.generated.IOError;
+import org.apache.hadoop.hbase.thrift.generated.IllegalArgument;
+import org.apache.hadoop.hbase.thrift.generated.Mutation;
+import org.apache.hadoop.hbase.thrift.generated.TAppend;
+import org.apache.hadoop.hbase.thrift.generated.TCell;
+import org.apache.hadoop.hbase.thrift.generated.TIncrement;
+import org.apache.hadoop.hbase.thrift.generated.TRegionInfo;
+import org.apache.hadoop.hbase.thrift.generated.TRowResult;
+import org.apache.hadoop.hbase.thrift.generated.TScan;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.thrift.TException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.base.Throwables;
+
+/**
+ * The HBaseServiceHandler is a glue object that connects Thrift RPC calls to the
+ * HBase client API primarily defined in the Admin and Table objects.
+ */
+@InterfaceAudience.Private
+@SuppressWarnings("deprecation")
+public class ThriftHBaseServiceHandler extends HBaseServiceHandler implements Hbase.Iface {
+  private static final Logger LOG = LoggerFactory.getLogger(ThriftHBaseServiceHandler.class);
+
+  public static final int HREGION_VERSION = 1;
+
+  // nextScannerId and scannerMap are used to manage scanner state
+  private int nextScannerId = 0;
+  private HashMap<Integer, ResultScannerWrapper> scannerMap;
+  IncrementCoalescer coalescer;
+
+  /**
+   * Returns a list of all the column families for a given Table.
+   *
+   * @param table table
+   * @throws IOException
+   */
+  byte[][] getAllColumns(Table table) throws IOException {
+    HColumnDescriptor[] cds = table.getTableDescriptor().getColumnFamilies();
+    byte[][] columns = new byte[cds.length][];
+    for (int i = 0; i < cds.length; i++) {
+      columns[i] = Bytes.add(cds[i].getName(),
+          KeyValue.COLUMN_FAMILY_DELIM_ARRAY);
+    }
+    return columns;
+  }
+
+
+  /**
+   * Assigns a unique ID to the scanner and adds the mapping to an internal
+   * hash-map.
+   *
+   * @param scanner the {@link ResultScanner} to add
+   * @return integer scanner id
+   */
+  protected synchronized int addScanner(ResultScanner scanner, boolean sortColumns) {
+    int id = nextScannerId++;
+    ResultScannerWrapper resultScannerWrapper =
+        new ResultScannerWrapper(scanner, sortColumns);
+    scannerMap.put(id, resultScannerWrapper);
+    return id;
+  }
+
+  /**
+   * Returns the scanner associated with the specified ID.
+   *
+   * @param id the ID of the scanner to get
+   * @return a Scanner, or null if ID was invalid.
+   */
+  private synchronized ResultScannerWrapper getScanner(int id) {
+    return scannerMap.get(id);
+  }
+
+  /**
+   * Removes the scanner associated with the specified ID from the internal
+   * id-&gt;scanner hash-map.
+   *
+   * @param id the ID of the scanner to remove
+   * @return a Scanner, or null if ID was invalid.
+   */
+  private synchronized ResultScannerWrapper removeScanner(int id) {
+    return scannerMap.remove(id);
+  }
+
+  protected ThriftHBaseServiceHandler(final Configuration c,
+      final UserProvider userProvider) throws IOException {
+    super(c, userProvider);
+    scannerMap = new HashMap<>();
+    this.coalescer = new IncrementCoalescer(this);
+  }
+
+
+  @Override
+  public void enableTable(ByteBuffer tableName) throws IOError {
+    try{
+      getAdmin().enableTable(getTableName(tableName));
+    } catch (IOException e) {
+      LOG.warn(e.getMessage(), e);
+      throw getIOError(e);
+    }
+  }
+
+  @Override
+  public void disableTable(ByteBuffer tableName) throws IOError{
+    try{
+      getAdmin().disableTable(getTableName(tableName));
+    } catch (IOException e) {
+      LOG.warn(e.getMessage(), e);
+      throw getIOError(e);
+    }
+  }
+
+  @Override
+  public boolean isTableEnabled(ByteBuffer tableName) throws IOError {
+    try {
+      return this.connectionCache.getAdmin().isTableEnabled(getTableName(tableName));
+    } catch (IOException e) {
+      LOG.warn(e.getMessage(), e);
+      throw getIOError(e);
+    }
+  }
+
+  // ThriftServerRunner.compact should be deprecated and replaced with methods specific to
+  // table and region.
+  @Override
+  public void compact(ByteBuffer tableNameOrRegionName) throws IOError {
+    try {
+      try {
+        getAdmin().compactRegion(getBytes(tableNameOrRegionName));
+      } catch (IllegalArgumentException e) {
+        // Invalid region, try table
+        getAdmin().compact(TableName.valueOf(getBytes(tableNameOrRegionName)));
+      }
+    } catch (IOException e) {
+      LOG.warn(e.getMessage(), e);
+      throw getIOError(e);
+    }
+  }
+
+  // ThriftServerRunner.majorCompact should be deprecated and replaced with methods specific
+  // to table and region.
+  @Override
+  public void majorCompact(ByteBuffer tableNameOrRegionName) throws IOError {
+    try {
+      try {
+        getAdmin().compactRegion(getBytes(tableNameOrRegionName));
+      } catch (IllegalArgumentException e) {
+        // Invalid region, try table
+        getAdmin().compact(TableName.valueOf(getBytes(tableNameOrRegionName)));
+      }
+    } catch (IOException e) {
+      LOG.warn(e.getMessage(), e);
+      throw getIOError(e);
+    }
+  }
+
+  @Override
+  public List<ByteBuffer> getTableNames() throws IOError {
+    try {
+      TableName[] tableNames = this.getAdmin().listTableNames();
+      ArrayList<ByteBuffer> list = new ArrayList<>(tableNames.length);
+      for (TableName tableName : tableNames) {
+        list.add(ByteBuffer.wrap(tableName.getName()));
+      }
+      return list;
+    } catch (IOException e) {
+      LOG.warn(e.getMessage(), e);
+      throw getIOError(e);
+    }
+  }
+
+  /**
+   * @return the list of regions in the given table, or an empty list if the table does not exist
+   */
+  @Override
+  public List<TRegionInfo> getTableRegions(ByteBuffer tableName) throws IOError {
+    try (RegionLocator locator = connectionCache.getRegionLocator(getBytes(tableName))) {
+      List<HRegionLocation> regionLocations = locator.getAllRegionLocations();
+      List<TRegionInfo> results = new ArrayList<>(regionLocations.size());
+      for (HRegionLocation regionLocation : regionLocations) {
+        RegionInfo info = regionLocation.getRegionInfo();
+        ServerName serverName = regionLocation.getServerName();
+        TRegionInfo region = new TRegionInfo();
+        region.serverName = ByteBuffer.wrap(
+            Bytes.toBytes(serverName.getHostname()));
+        region.port = serverName.getPort();
+        region.startKey = ByteBuffer.wrap(info.getStartKey());
+        region.endKey = ByteBuffer.wrap(info.getEndKey());
+        region.id = info.getRegionId();
+        region.name = ByteBuffer.wrap(info.getRegionName());
+        region.version = HREGION_VERSION; // HRegion now not versioned, PB encoding used
+        results.add(region);
+      }
+      return results;
+    } catch (TableNotFoundException e) {
+      // Return empty list for non-existing table
+      return Collections.emptyList();
+    } catch (IOException e){
+      LOG.warn(e.getMessage(), e);
+      throw getIOError(e);
+    }
+  }
+
+  @Override
+  public List<TCell> get(
+      ByteBuffer tableName, ByteBuffer row, ByteBuffer column,
+      Map<ByteBuffer, ByteBuffer> attributes)
+      throws IOError {
+    byte [][] famAndQf = CellUtil.parseColumn(getBytes(column));
+    if (famAndQf.length == 1) {
+      return get(tableName, row, famAndQf[0], null, attributes);
+    }
+    if (famAndQf.length == 2) {
+      return get(tableName, row, famAndQf[0], famAndQf[1], attributes);
+    }
+    throw new IllegalArgumentException("Invalid familyAndQualifier provided.");
+  }
+
+  /**
+   * Note: this internal interface is slightly different from public APIs in regard to handling
+   * of the qualifier. Here we differ from the public Java API in that null != byte[0]. Rather,
+   * we respect qual == null as a request for the entire column family. The caller (
+   * {@link #get(ByteBuffer, ByteBuffer, ByteBuffer, Map)}) interface IS consistent in that the
+   * column is parse like normal.
+   */
+  protected List<TCell> get(ByteBuffer tableName,
+      ByteBuffer row,
+      byte[] family,
+      byte[] qualifier,
+      Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
+    Table table = null;
+    try {
+      table = getTable(tableName);
+      Get get = new Get(getBytes(row));
+      addAttributes(get, attributes);
+      if (qualifier == null) {
+        get.addFamily(family);
+      } else {
+        get.addColumn(family, qualifier);
+      }
+      Result result = table.get(get);
+      return ThriftUtilities.cellFromHBase(result.rawCells());
+    } catch (IOException e) {
+      LOG.warn(e.getMessage(), e);
+      throw getIOError(e);
+    } finally {
+      closeTable(table);
+    }
+  }
+
+  @Override
+  public List<TCell> getVer(ByteBuffer tableName, ByteBuffer row, ByteBuffer column,
+      int numVersions, Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
+    byte [][] famAndQf = CellUtil.parseColumn(getBytes(column));
+    if(famAndQf.length == 1) {
+      return getVer(tableName, row, famAndQf[0], null, numVersions, attributes);
+    }
+    if (famAndQf.length == 2) {
+      return getVer(tableName, row, famAndQf[0], famAndQf[1], numVersions, attributes);
+    }
+    throw new IllegalArgumentException("Invalid familyAndQualifier provided.");
+
+  }
+
+  /**
+   * Note: this public interface is slightly different from public Java APIs in regard to
+   * handling of the qualifier. Here we differ from the public Java API in that null != byte[0].
+   * Rather, we respect qual == null as a request for the entire column family. If you want to
+   * access the entire column family, use
+   * {@link #getVer(ByteBuffer, ByteBuffer, ByteBuffer, int, Map)} with a {@code column} value
+   * that lacks a {@code ':'}.
+   */
+  public List<TCell> getVer(ByteBuffer tableName, ByteBuffer row, byte[] family,
+      byte[] qualifier, int numVersions, Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
+
+    Table table = null;
+    try {
+      table = getTable(tableName);
+      Get get = new Get(getBytes(row));
+      addAttributes(get, attributes);
+      if (null == qualifier) {
+        get.addFamily(family);
+      } else {
+        get.addColumn(family, qualifier);
+      }
+      get.setMaxVersions(numVersions);
+      Result result = table.get(get);
+      return ThriftUtilities.cellFromHBase(result.rawCells());
+    } catch (IOException e) {
+      LOG.warn(e.getMessage(), e);
+      throw getIOError(e);
+    } finally{
+      closeTable(table);
+    }
+  }
+
+  @Override
+  public List<TCell> getVerTs(ByteBuffer tableName, ByteBuffer row, ByteBuffer column,
+      long timestamp, int numVersions, Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
+    byte [][] famAndQf = CellUtil.parseColumn(getBytes(column));
+    if (famAndQf.length == 1) {
+      return getVerTs(tableName, row, famAndQf[0], null, timestamp, numVersions, attributes);
+    }
+    if (famAndQf.length == 2) {
+      return getVerTs(tableName, row, famAndQf[0], famAndQf[1], timestamp, numVersions,
+          attributes);
+    }
+    throw new IllegalArgumentException("Invalid familyAndQualifier provided.");
+  }
+
+  /**
+   * Note: this internal interface is slightly different from public APIs in regard to handling
+   * of the qualifier. Here we differ from the public Java API in that null != byte[0]. Rather,
+   * we respect qual == null as a request for the entire column family. The caller (
+   * {@link #getVerTs(ByteBuffer, ByteBuffer, ByteBuffer, long, int, Map)}) interface IS
+   * consistent in that the column is parse like normal.
+   */
+  protected List<TCell> getVerTs(ByteBuffer tableName, ByteBuffer row, byte[] family,
+      byte[] qualifier, long timestamp, int numVersions, Map<ByteBuffer, ByteBuffer> attributes)
+      throws IOError {
+
+    Table table = null;
+    try {
+      table = getTable(tableName);
+      Get get = new Get(getBytes(row));
+      addAttributes(get, attributes);
+      if (null == qualifier) {
+        get.addFamily(family);
+      } else {
+        get.addColumn(family, qualifier);
+      }
+      get.setTimeRange(0, timestamp);
+      get.setMaxVersions(numVersions);
+      Result result = table.get(get);
+      return ThriftUtilities.cellFromHBase(result.rawCells());
+    } catch (IOException e) {
+      LOG.warn(e.getMessage(), e);
+      throw getIOError(e);
+    } finally{
+      closeTable(table);
+    }
+  }
+
+  @Override
+  public List<TRowResult> getRow(ByteBuffer tableName, ByteBuffer row,
+      Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
+    return getRowWithColumnsTs(tableName, row, null,
+        HConstants.LATEST_TIMESTAMP,
+        attributes);
+  }
+
+  @Override
+  public List<TRowResult> getRowWithColumns(ByteBuffer tableName,
+      ByteBuffer row,
+      List<ByteBuffer> columns,
+      Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
+    return getRowWithColumnsTs(tableName, row, columns,
+        HConstants.LATEST_TIMESTAMP,
+        attributes);
+  }
+
+  @Override
+  public List<TRowResult> getRowTs(ByteBuffer tableName, ByteBuffer row,
+      long timestamp, Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
+    return getRowWithColumnsTs(tableName, row, null,
+        timestamp, attributes);
+  }
+
+  @Override
+  public List<TRowResult> getRowWithColumnsTs(
+      ByteBuffer tableName, ByteBuffer row, List<ByteBuffer> columns,
+      long timestamp, Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
+
+    Table table = null;
+    try {
+      table = getTable(tableName);
+      if (columns == null) {
+        Get get = new Get(getBytes(row));
+        addAttributes(get, attributes);
+        get.setTimeRange(0, timestamp);
+        Result result = table.get(get);
+        return ThriftUtilities.rowResultFromHBase(result);
+      }
+      Get get = new Get(getBytes(row));
+      addAttributes(get, attributes);
+      for(ByteBuffer column : columns) {
+        byte [][] famAndQf = CellUtil.parseColumn(getBytes(column));
+        if (famAndQf.length == 1) {
+          get.addFamily(famAndQf[0]);
+        } else {
+          get.addColumn(famAndQf[0], famAndQf[1]);
+        }
+      }
+      get.setTimeRange(0, timestamp);
+      Result result = table.get(get);
+      return ThriftUtilities.rowResultFromHBase(result);
+    } catch (IOException e) {
+      LOG.warn(e.getMessage(), e);
+      throw getIOError(e);
+    } finally{
+      closeTable(table);
+    }
+  }
+
+  @Override
+  public List<TRowResult> getRows(ByteBuffer tableName,
+      List<ByteBuffer> rows,
+      Map<ByteBuffer, ByteBuffer> attributes)
+      throws IOError {
+    return getRowsWithColumnsTs(tableName, rows, null,
+        HConstants.LATEST_TIMESTAMP,
+        attributes);
+  }
+
+  @Override
+  public List<TRowResult> getRowsWithColumns(ByteBuffer tableName,
+      List<ByteBuffer> rows,
+      List<ByteBuffer> columns,
+      Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
+    return getRowsWithColumnsTs(tableName, rows, columns,
+        HConstants.LATEST_TIMESTAMP,
+        attributes);
+  }
+
+  @Override
+  public List<TRowResult> getRowsTs(ByteBuffer tableName,
+      List<ByteBuffer> rows,
+      long timestamp,
+      Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
+    return getRowsWithColumnsTs(tableName, rows, null,
+        timestamp, attributes);
+  }
+
+  @Override
+  public List<TRowResult> getRowsWithColumnsTs(ByteBuffer tableName,
+      List<ByteBuffer> rows,
+      List<ByteBuffer> columns, long timestamp,
+      Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
+
+    Table table= null;
+    try {
+      List<Get> gets = new ArrayList<>(rows.size());
+      table = getTable(tableName);
+      if (metrics != null) {
+        metrics.incNumRowKeysInBatchGet(rows.size());
+      }
+      for (ByteBuffer row : rows) {
+        Get get = new Get(getBytes(row));
+        addAttributes(get, attributes);
+        if (columns != null) {
+
+          for(ByteBuffer column : columns) {
+            byte [][] famAndQf = CellUtil.parseColumn(getBytes(column));
+            if (famAndQf.length == 1) {
+              get.addFamily(famAndQf[0]);
+            } else {
+              get.addColumn(famAndQf[0], famAndQf[1]);
+            }
+          }
+        }
+        get.setTimeRange(0, timestamp);
+        gets.add(get);
+      }
+      Result[] result = table.get(gets);
+      return ThriftUtilities.rowResultFromHBase(result);
+    } catch (IOException e) {
+      LOG.warn(e.getMessage(), e);
+      throw getIOError(e);
+    } finally{
+      closeTable(table);
+    }
+  }
+
+  @Override
+  public void deleteAll(
+      ByteBuffer tableName, ByteBuffer row, ByteBuffer column,
+      Map<ByteBuffer, ByteBuffer> attributes)
+      throws IOError {
+    deleteAllTs(tableName, row, column, HConstants.LATEST_TIMESTAMP,
+        attributes);
+  }
+
+  @Override
+  public void deleteAllTs(ByteBuffer tableName,
+      ByteBuffer row,
+      ByteBuffer column,
+      long timestamp, Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
+    Table table = null;
+    try {
+      table = getTable(tableName);
+      Delete delete  = new Delete(getBytes(row));
+      addAttributes(delete, attributes);
+      byte [][] famAndQf = CellUtil.parseColumn(getBytes(column));
+      if (famAndQf.length == 1) {
+        delete.addFamily(famAndQf[0], timestamp);
+      } else {
+        delete.addColumns(famAndQf[0], famAndQf[1], timestamp);
+      }
+      table.delete(delete);
+
+    } catch (IOException e) {
+      LOG.warn(e.getMessage(), e);
+      throw getIOError(e);
+    } finally {
+      closeTable(table);
+    }
+  }
+
+  @Override
+  public void deleteAllRow(
+      ByteBuffer tableName, ByteBuffer row,
+      Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
+    deleteAllRowTs(tableName, row, HConstants.LATEST_TIMESTAMP, attributes);
+  }
+
+  @Override
+  public void deleteAllRowTs(
+      ByteBuffer tableName, ByteBuffer row, long timestamp,
+      Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
+    Table table = null;
+    try {
+      table = getTable(tableName);
+      Delete delete  = new Delete(getBytes(row), timestamp);
+      addAttributes(delete, attributes);
+      table.delete(delete);
+    } catch (IOException e) {
+      LOG.warn(e.getMessage(), e);
+      throw getIOError(e);
+    } finally {
+      closeTable(table);
+    }
+  }
+
+  @Override
+  public void createTable(ByteBuffer in_tableName,
+      List<ColumnDescriptor> columnFamilies) throws IOError, IllegalArgument, AlreadyExists {
+    TableName tableName = getTableName(in_tableName);
+    try {
+      if (getAdmin().tableExists(tableName)) {
+        throw new AlreadyExists("table name already in use");
+      }
+      HTableDescriptor desc = new HTableDescriptor(tableName);
+      for (ColumnDescriptor col : columnFamilies) {
+        HColumnDescriptor colDesc = ThriftUtilities.colDescFromThrift(col);
+        desc.addFamily(colDesc);
+      }
+      getAdmin().createTable(desc);
+    } catch (IOException e) {
+      LOG.warn(e.getMessage(), e);
+      throw getIOError(e);
+    } catch (IllegalArgumentException e) {
+      LOG.warn(e.getMessage(), e);
+      throw new IllegalArgument(Throwables.getStackTraceAsString(e));
+    }
+  }
+
+  private static TableName getTableName(ByteBuffer buffer) {
+    return TableName.valueOf(getBytes(buffer));
+  }
+
+  @Override
+  public void deleteTable(ByteBuffer in_tableName) throws IOError {
+    TableName tableName = getTableName(in_tableName);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("deleteTable: table={}", tableName);
+    }
+    try {
+      if (!getAdmin().tableExists(tableName)) {
+        throw new IOException("table does not exist");
+      }
+      getAdmin().deleteTable(tableName);
+    } catch (IOException e) {
+      LOG.warn(e.getMessage(), e);
+      throw getIOError(e);
+    }
+  }
+
+  @Override
+  public void mutateRow(ByteBuffer tableName, ByteBuffer row,
+      List<Mutation> mutations, Map<ByteBuffer, ByteBuffer> attributes)
+      throws IOError, IllegalArgument {
+    mutateRowTs(tableName, row, mutations, HConstants.LATEST_TIMESTAMP, attributes);
+  }
+
+  @Override
+  public void mutateRowTs(ByteBuffer tableName, ByteBuffer row,
+      List<Mutation> mutations, long timestamp,
+      Map<ByteBuffer, ByteBuffer> attributes)
+      throws IOError, IllegalArgument {
+    Table table = null;
+    try {
+      table = getTable(tableName);
+      Put put = new Put(getBytes(row), timestamp);
+      addAttributes(put, attributes);
+
+      Delete delete = new Delete(getBytes(row));
+      addAttributes(delete, attributes);
+      if (metrics != null) {
+        metrics.incNumRowKeysInBatchMutate(mutations.size());
+      }
+
+      // I apologize for all this mess :)
+      CellBuilder builder = CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY);
+      for (Mutation m : mutations) {
+        byte[][] famAndQf = CellUtil.parseColumn(getBytes(m.column));
+        if (m.isDelete) {
+          if (famAndQf.length == 1) {
+            delete.addFamily(famAndQf[0], timestamp);
+          } else {
+            delete.addColumns(famAndQf[0], famAndQf[1], timestamp);
+          }
+          delete.setDurability(m.writeToWAL ? Durability.SYNC_WAL : Durability.SKIP_WAL);
+        } else {
+          if(famAndQf.length == 1) {
+            LOG.warn("No column qualifier specified. Delete is the only mutation supported "
+                + "over the whole column family.");
+          } else {
+            put.add(builder.clear()
+                .setRow(put.getRow())
+                .setFamily(famAndQf[0])
+                .setQualifier(famAndQf[1])
+                .setTimestamp(put.getTimestamp())
+                .setType(Cell.Type.Put)
+                .setValue(m.value != null ? getBytes(m.value)
+                    : HConstants.EMPTY_BYTE_ARRAY)
+                .build());
+          }
+          put.setDurability(m.writeToWAL ? Durability.SYNC_WAL : Durability.SKIP_WAL);
+        }
+      }
+      if (!delete.isEmpty()) {
+        table.delete(delete);
+      }
+      if (!put.isEmpty()) {
+        table.put(put);
+      }
+    } catch (IOException e) {
+      LOG.warn(e.getMessage(), e);
+      throw getIOError(e);
+    } catch (IllegalArgumentException e) {
+      LOG.warn(e.getMessage(), e);
+      throw new IllegalArgument(Throwables.getStackTraceAsString(e));
+    } finally{
+      closeTable(table);
+    }
+  }
+
+  @Override
+  public void mutateRows(ByteBuffer tableName, List<BatchMutation> rowBatches,
+      Map<ByteBuffer, ByteBuffer> attributes)
+      throws IOError, IllegalArgument, TException {
+    mutateRowsTs(tableName, rowBatches, HConstants.LATEST_TIMESTAMP, attributes);
+  }
+
+  @Override
+  public void mutateRowsTs(
+      ByteBuffer tableName, List<BatchMutation> rowBatches, long timestamp,
+      Map<ByteBuffer, ByteBuffer> attributes)
+      throws IOError, IllegalArgument, TException {
+    List<Put> puts = new ArrayList<>();
+    List<Delete> deletes = new ArrayList<>();
+    CellBuilder builder = CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY);
+    for (BatchMutation batch : rowBatches) {
+      byte[] row = getBytes(batch.row);
+      List<Mutation> mutations = batch.mutations;
+      Delete delete = new Delete(row);
+      addAttributes(delete, attributes);
+      Put put = new Put(row, timestamp);
+      addAttributes(put, attributes);
+      for (Mutation m : mutations) {
+        byte[][] famAndQf = CellUtil.parseColumn(getBytes(m.column));
+        if (m.isDelete) {
+          // no qualifier, family only.
+          if (famAndQf.length == 1) {
+            delete.addFamily(famAndQf[0], timestamp);
+          } else {
+            delete.addColumns(famAndQf[0], famAndQf[1], timestamp);
+          }
+          delete.setDurability(m.writeToWAL ? Durability.SYNC_WAL
+              : Durability.SKIP_WAL);
+        } else {
+          if (famAndQf.length == 1) {
+            LOG.warn("No column qualifier specified. Delete is the only mutation supported "
+                + "over the whole column family.");
+          }
+          if (famAndQf.length == 2) {
+            try {
+              put.add(builder.clear()
+                  .setRow(put.getRow())
+                  .setFamily(famAndQf[0])
+                  .setQualifier(famAndQf[1])
+                  .setTimestamp(put.getTimestamp())
+                  .setType(Cell.Type.Put)
+                  .setValue(m.value != null ? getBytes(m.value)
+                      : HConstants.EMPTY_BYTE_ARRAY)
+                  .build());
+            } catch (IOException e) {
+              throw new IllegalArgumentException(e);
+            }
+          } else {
+            throw new IllegalArgumentException("Invalid famAndQf provided.");
+          }
+          put.setDurability(m.writeToWAL ? Durability.SYNC_WAL : Durability.SKIP_WAL);
+        }
+      }
+      if (!delete.isEmpty()) {
+        deletes.add(delete);
+      }
+      if (!put.isEmpty()) {
+        puts.add(put);
+      }
+    }
+
+    Table table = null;
+    try {
+      table = getTable(tableName);
+      if (!puts.isEmpty()) {
+        table.put(puts);
+      }
+      if (!deletes.isEmpty()) {
+        table.delete(deletes);
+      }
+    } catch (IOException e) {
+      LOG.warn(e.getMessage(), e);
+      throw getIOError(e);
+    } catch (IllegalArgumentException e) {
+      LOG.warn(e.getMessage(), e);
+      throw new IllegalArgument(Throwables.getStackTraceAsString(e));
+    } finally{
+      closeTable(table);
+    }
+  }
+
+  @Override
+  public long atomicIncrement(
+      ByteBuffer tableName, ByteBuffer row, ByteBuffer column, long amount)
+      throws IOError, IllegalArgument, TException {
+    byte [][] famAndQf = CellUtil.parseColumn(getBytes(column));
+    if(famAndQf.length == 1) {
+      return atomicIncrement(tableName, row, famAndQf[0], HConstants.EMPTY_BYTE_ARRAY, amount);
+    }
+    return atomicIncrement(tableName, row, famAndQf[0], famAndQf[1], amount);
+  }
+
+  protected long atomicIncrement(ByteBuffer tableName, ByteBuffer row,
+      byte [] family, byte [] qualifier, long amount)
+      throws IOError, IllegalArgument, TException {
+    Table table = null;
+    try {
+      table = getTable(tableName);
+      return table.incrementColumnValue(
+          getBytes(row), family, qualifier, amount);
+    } catch (IOException e) {
+      LOG.warn(e.getMessage(), e);
+      throw getIOError(e);
+    } finally {
+      closeTable(table);
+    }
+  }
+
+  @Override
+  public void scannerClose(int id) throws IOError, IllegalArgument {
+    LOG.debug("scannerClose: id={}", id);
+    ResultScannerWrapper resultScannerWrapper = getScanner(id);
+    if (resultScannerWrapper == null) {
+      LOG.warn("scanner ID is invalid");
+      throw new IllegalArgument("scanner ID is invalid");
+    }
+    resultScannerWrapper.getScanner().close();
+    removeScanner(id);
+  }
+
+  @Override
+  public List<TRowResult> scannerGetList(int id,int nbRows)
+      throws IllegalArgument, IOError {
+    LOG.debug("scannerGetList: id={}", id);
+    ResultScannerWrapper resultScannerWrapper = getScanner(id);
+    if (null == resultScannerWrapper) {
+      String message = "scanner ID is invalid";
+      LOG.warn(message);
+      throw new IllegalArgument("scanner ID is invalid");
+    }
+
+    Result [] results;
+    try {
+      results = resultScannerWrapper.getScanner().next(nbRows);
+      if (null == results) {
+        return new ArrayList<>();
+      }
+    } catch (IOException e) {
+      LOG.warn(e.getMessage(), e);
+      throw getIOError(e);
+    }
+    return ThriftUtilities.rowResultFromHBase(results, resultScannerWrapper.isColumnSorted());
+  }
+
+  @Override
+  public List<TRowResult> scannerGet(int id) throws IllegalArgument, IOError {
+    return scannerGetList(id,1);
+  }
+
+  @Override
+  public int scannerOpenWithScan(ByteBuffer tableName, TScan tScan,
+      Map<ByteBuffer, ByteBuffer> attributes)
+      throws IOError {
+
+    Table table = null;
+    try {
+      table = getTable(tableName);
+      Scan scan = new Scan();
+      addAttributes(scan, attributes);
+      if (tScan.isSetStartRow()) {
+        scan.setStartRow(tScan.getStartRow());
+      }
+      if (tScan.isSetStopRow()) {
+        scan.setStopRow(tScan.getStopRow());
+      }
+      if (tScan.isSetTimestamp()) {
+        scan.setTimeRange(0, tScan.getTimestamp());
+      }
+      if (tScan.isSetCaching()) {
+        scan.setCaching(tScan.getCaching());
+      }
+      if (tScan.isSetBatchSize()) {
+        scan.setBatch(tScan.getBatchSize());
+      }
+      if (tScan.isSetColumns() && !tScan.getColumns().isEmpty()) {
+        for(ByteBuffer column : tScan.getColumns()) {
+          byte [][] famQf = CellUtil.parseColumn(getBytes(column));
+          if(famQf.length == 1) {
+            scan.addFamily(famQf[0]);
+          } else {
+            scan.addColumn(famQf[0], famQf[1]);
+          }
+        }
+      }
+      if (tScan.isSetFilterString()) {
+        ParseFilter parseFilter = new ParseFilter();
+        scan.setFilter(
+            parseFilter.parseFilterString(tScan.getFilterString()));
+      }
+      if (tScan.isSetReversed()) {
+        scan.setReversed(tScan.isReversed());
+      }
+      if (tScan.isSetCacheBlocks()) {
+        scan.setCacheBlocks(tScan.isCacheBlocks());
+      }
+      return addScanner(table.getScanner(scan), tScan.sortColumns);
+    } catch (IOException e) {
+      LOG.warn(e.getMessage(), e);
+      throw getIOError(e);
+    } finally{
+      closeTable(table);
+    }
+  }
+
+  @Override
+  public int scannerOpen(ByteBuffer tableName, ByteBuffer startRow,
+      List<ByteBuffer> columns,
+      Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
+
+    Table table = null;
+    try {
+      table = getTable(tableName);
+      Scan scan = new Scan(getBytes(startRow));
+      addAttributes(scan, attributes);
+      if(columns != null && !columns.isEmpty()) {
+        for(ByteBuffer column : columns) {
+          byte [][] famQf = CellUtil.parseColumn(getBytes(column));
+          if(famQf.length == 1) {
+            scan.addFamily(famQf[0]);
+          } else {
+            scan.addColumn(famQf[0], famQf[1]);
+          }
+        }
+      }
+      return addScanner(table.getScanner(scan), false);
+    } catch (IOException e) {
+      LOG.warn(e.getMessage(), e);
+      throw getIOError(e);
+    } finally{
+      closeTable(table);
+    }
+  }
+
+  @Override
+  public int scannerOpenWithStop(ByteBuffer tableName, ByteBuffer startRow,
+      ByteBuffer stopRow, List<ByteBuffer> columns,
+      Map<ByteBuffer, ByteBuffer> attributes)
+      throws IOError, TException {
+
+    Table table = null;
+    try {
+      table = getTable(tableName);
+      Scan scan = new Scan(getBytes(startRow), getBytes(stopRow));
+      addAttributes(scan, attributes);
+      if(columns != null && !columns.isEmpty()) {
+        for(ByteBuffer column : columns) {
+          byte [][] famQf = CellUtil.parseColumn(getBytes(column));
+          if(famQf.length == 1) {
+            scan.addFamily(famQf[0]);
+          } else {
+            scan.addColumn(famQf[0], famQf[1]);
+          }
+        }
+      }
+      return addScanner(table.getScanner(scan), false);
+    } catch (IOException e) {
+      LOG.warn(e.getMessage(), e);
+      throw getIOError(e);
+    } finally{
+      closeTable(table);
+    }
+  }
+
+  @Override
+  public int scannerOpenWithPrefix(ByteBuffer tableName,
+      ByteBuffer startAndPrefix,
+      List<ByteBuffer> columns,
+      Map<ByteBuffer, ByteBuffer> attributes)
+      throws IOError, TException {
+
+    Table table = null;
+    try {
+      table = getTable(tableName);
+      Scan scan = new Scan(getBytes(startAndPrefix));
+      addAttributes(scan, attributes);
+      Filter f = new WhileMatchFilter(
+          new PrefixFilter(getBytes(startAndPrefix)));
+      scan.setFilter(f);
+      if (columns != null && !columns.isEmpty()) {
+        for(ByteBuffer column : columns) {
+          byte [][] famQf = CellUtil.parseColumn(getBytes(column));
+          if(famQf.length == 1) {
+            scan.addFamily(famQf[0]);
+          } else {
+            scan.addColumn(famQf[0], famQf[1]);
+          }
+        }
+      }
+      return addScanner(table.getScanner(scan), false);
+    } catch (IOException e) {
+      LOG.warn(e.getMessage(), e);
+      throw getIOError(e);
+    } finally{
+      closeTable(table);
+    }
+  }
+
+  @Override
+  public int scannerOpenTs(ByteBuffer tableName, ByteBuffer startRow,
+      List<ByteBuffer> columns, long timestamp,
+      Map<ByteBuffer, ByteBuffer> attributes) throws IOError, TException {
+
+    Table table = null;
+    try {
+      table = getTable(tableName);
+      Scan scan = new Scan(getBytes(startRow));
+      addAttributes(scan, attributes);
+      scan.setTimeRange(0, timestamp);
+      if (columns != null && !columns.isEmpty()) {
+        for (ByteBuffer column : columns) {
+          byte [][] famQf = CellUtil.parseColumn(getBytes(column));
+          if(famQf.length == 1) {
+            scan.addFamily(famQf[0]);
+          } else {
+            scan.addColumn(famQf[0], famQf[1]);
+          }
+        }
+      }
+      return addScanner(table.getScanner(scan), false);
+    } catch (IOException e) {
+      LOG.warn(e.getMessage(), e);
+      throw getIOError(e);
+    } finally{
+      closeTable(table);
+    }
+  }
+
+  @Override
+  public int scannerOpenWithStopTs(ByteBuffer tableName, ByteBuffer startRow,
+      ByteBuffer stopRow, List<ByteBuffer> columns, long timestamp,
+      Map<ByteBuffer, ByteBuffer> attributes)
+      throws IOError, TException {
+
+    Table table = null;
+    try {
+      table = getTable(tableName);
+      Scan scan = new Scan(getBytes(startRow), getBytes(stopRow));
+      addAttributes(scan, attributes);
+      scan.setTimeRange(0, timestamp);
+      if (columns != null && !columns.isEmpty()) {
+        for (ByteBuffer column : columns) {
+          byte [][] famQf = CellUtil.parseColumn(getBytes(column));
+          if(famQf.length == 1) {
+            scan.addFamily(famQf[0]);
+          } else {
+            scan.addColumn(famQf[0], famQf[1]);
+          }
+        }
+      }
+      scan.setTimeRange(0, timestamp);
+      return addScanner(table.getScanner(scan), false);
+    } catch (IOException e) {
+      LOG.warn(e.getMessage(), e);
+      throw getIOError(e);
+    } finally{
+      closeTable(table);
+    }
+  }
+
+  @Override
+  public Map<ByteBuffer, ColumnDescriptor> getColumnDescriptors(
+      ByteBuffer tableName) throws IOError, TException {
+
+    Table table = null;
+    try {
+      TreeMap<ByteBuffer, ColumnDescriptor> columns = new TreeMap<>();
+
+      table = getTable(tableName);
+      HTableDescriptor desc = table.getTableDescriptor();
+
+      for (HColumnDescriptor e : desc.getFamilies()) {
+        ColumnDescriptor col = ThriftUtilities.colDescFromHbase(e);
+        columns.put(col.name, col);
+      }
+      return columns;
+    } catch (IOException e) {
+      LOG.warn(e.getMessage(), e);
+      throw getIOError(e);
+    } finally {
+      closeTable(table);
+    }
+  }
+
+  private void closeTable(Table table) throws IOError {
+    try{
+      if(table != null){
+        table.close();
+      }
+    } catch (IOException e){
+      LOG.error(e.getMessage(), e);
+      throw getIOError(e);
+    }
+  }
+
+  @Override
+  public TRegionInfo getRegionInfo(ByteBuffer searchRow) throws IOError {
+    try {
+      byte[] row = getBytes(searchRow);
+      Result startRowResult = getReverseScanResult(TableName.META_TABLE_NAME.getName(), row,
+          HConstants.CATALOG_FAMILY);
+
+      if (startRowResult == null) {
+        throw new IOException("Cannot find row in "+ TableName.META_TABLE_NAME+", row="
+            + Bytes.toStringBinary(row));
+      }
+
+      // find region start and end keys
+      RegionInfo regionInfo = MetaTableAccessor.getRegionInfo(startRowResult);
+      if (regionInfo == null) {
+        throw new IOException("RegionInfo REGIONINFO was null or " +
+            " empty in Meta for row="
+            + Bytes.toStringBinary(row));
+      }
+      TRegionInfo region = new TRegionInfo();
+      region.setStartKey(regionInfo.getStartKey());
+      region.setEndKey(regionInfo.getEndKey());
+      region.id = regionInfo.getRegionId();
+      region.setName(regionInfo.getRegionName());
+      region.version = HREGION_VERSION; // version not used anymore, PB encoding used.
+
+      // find region assignment to server
+      ServerName serverName = MetaTableAccessor.getServerName(startRowResult, 0);
+      if (serverName != null) {
+        region.setServerName(Bytes.toBytes(serverName.getHostname()));
+        region.port = serverName.getPort();
+      }
+      return region;
+    } catch (IOException e) {
+      LOG.warn(e.getMessage(), e);
+      throw getIOError(e);
+    }
+  }
+
+  private Result getReverseScanResult(byte[] tableName, byte[] row, byte[] family)
+      throws IOException {
+    Scan scan = new Scan(row);
+    scan.setReversed(true);
+    scan.addFamily(family);
+    scan.setStartRow(row);
+    try (Table table = getTable(tableName);
+         ResultScanner scanner = table.getScanner(scan)) {
+      return scanner.next();
+    }
+  }
+
+  @Override
+  public void increment(TIncrement tincrement) throws IOError, TException {
+
+    if (tincrement.getRow().length == 0 || tincrement.getTable().length == 0) {
+      throw new TException("Must supply a table and a row key; can't increment");
+    }
+
+    if (conf.getBoolean(COALESCE_INC_KEY, false)) {
+      this.coalescer.queueIncrement(tincrement);
+      return;
+    }
+
+    Table table = null;
+    try {
+      table = getTable(tincrement.getTable());
+      Increment inc = ThriftUtilities.incrementFromThrift(tincrement);
+      table.increment(inc);
+    } catch (IOException e) {
+      LOG.warn(e.getMessage(), e);
+      throw getIOError(e);
+    } finally{
+      closeTable(table);
+    }
+  }
+
+  @Override
+  public void incrementRows(List<TIncrement> tincrements) throws IOError, TException {
+    if (conf.getBoolean(COALESCE_INC_KEY, false)) {
+      this.coalescer.queueIncrements(tincrements);
+      return;
+    }
+    for (TIncrement tinc : tincrements) {
+      increment(tinc);
+    }
+  }
+
+  @Override
+  public List<TCell> append(TAppend tappend) throws IOError, TException {
+    if (tappend.getRow().length == 0 || tappend.getTable().length == 0) {
+      throw new TException("Must supply a table and a row key; can't append");
+    }
+
+    Table table = null;
+    try {
+      table = getTable(tappend.getTable());
+      Append append = ThriftUtilities.appendFromThrift(tappend);
+      Result result = table.append(append);
+      return ThriftUtilities.cellFromHBase(result.rawCells());
+    } catch (IOException e) {
+      LOG.warn(e.getMessage(), e);
+      throw getIOError(e);
+    } finally{
+      closeTable(table);
+    }
+  }
+
+  @Override
+  public boolean checkAndPut(ByteBuffer tableName, ByteBuffer row, ByteBuffer column,
+      ByteBuffer value, Mutation mput, Map<ByteBuffer, ByteBuffer> attributes) throws IOError,
+      IllegalArgument, TException {
+    Put put;
+    try {
+      put = new Put(getBytes(row), HConstants.LATEST_TIMESTAMP);
+      addAttributes(put, attributes);
+
+      byte[][] famAndQf = CellUtil.parseColumn(getBytes(mput.column));
+      put.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY)
+          .setRow(put.getRow())
+          .setFamily(famAndQf[0])
+          .setQualifier(famAndQf[1])
+          .setTimestamp(put.getTimestamp())
+          .setType(Cell.Type.Put)
+          .setValue(mput.value != null ? getBytes(mput.value)
+              : HConstants.EMPTY_BYTE_ARRAY)
+          .build());
+      put.setDurability(mput.writeToWAL ? Durability.SYNC_WAL : Durability.SKIP_WAL);
+    } catch (IOException | IllegalArgumentException e) {
+      LOG.warn(e.getMessage(), e);
+      throw new IllegalArgument(Throwables.getStackTraceAsString(e));
+    }
+
+    Table table = null;
+    try {
+      table = getTable(tableName);
+      byte[][] famAndQf = CellUtil.parseColumn(getBytes(column));
+      Table.CheckAndMutateBuilder mutateBuilder =
+          table.checkAndMutate(getBytes(row), famAndQf[0]).qualifier(famAndQf[1]);
+      if (value != null) {
+        return mutateBuilder.ifEquals(getBytes(value)).thenPut(put);
+      } else {
+        return mutateBuilder.ifNotExists().thenPut(put);
+      }
+    } catch (IOException e) {
+      LOG.warn(e.getMessage(), e);
+      throw getIOError(e);
+    } catch (IllegalArgumentException e) {
+      LOG.warn(e.getMessage(), e);
+      throw new IllegalArgument(Throwables.getStackTraceAsString(e));
+    } finally {
+      closeTable(table);
+    }
+  }
+
+  private static IOError getIOError(Throwable throwable) {
+    IOError error = new IOErrorWithCause(throwable);
+    error.setMessage(Throwables.getStackTraceAsString(throwable));
+    return error;
+  }
+
+  /**
+   * Adds all the attributes into the Operation object
+   */
+  private static void addAttributes(OperationWithAttributes op,
+      Map<ByteBuffer, ByteBuffer> attributes) {
+    if (attributes == null || attributes.isEmpty()) {
+      return;
+    }
+    for (Map.Entry<ByteBuffer, ByteBuffer> entry : attributes.entrySet()) {
+      String name = Bytes.toStringBinary(getBytes(entry.getKey()));
+      byte[] value =  getBytes(entry.getValue());
+      op.setAttribute(name, value);
+    }
+  }
+
+  protected static class ResultScannerWrapper {
+
+    private final ResultScanner scanner;
+    private final boolean sortColumns;
+    public ResultScannerWrapper(ResultScanner resultScanner,
+        boolean sortResultColumns) {
+      scanner = resultScanner;
+      sortColumns = sortResultColumns;
+    }
+
+    public ResultScanner getScanner() {
+      return scanner;
+    }
+
+    public boolean isColumnSorted() {
+      return sortColumns;
+    }
+  }
+
+  public static class IOErrorWithCause extends IOError {
+    private final Throwable cause;
+    public IOErrorWithCause(Throwable cause) {
+      this.cause = cause;
+    }
+
+    @Override
+    public synchronized Throwable getCause() {
+      return cause;
+    }
+
+    @Override
+    public boolean equals(Object other) {
+      if (super.equals(other) &&
+          other instanceof IOErrorWithCause) {
+        Throwable otherCause = ((IOErrorWithCause) other).getCause();
+        if (this.getCause() != null) {
+          return otherCause != null && this.getCause().equals(otherCause);
+        } else {
+          return otherCause == null;
+        }
+      }
+      return false;
+    }
+
+    @Override
+    public int hashCode() {
+      int result = super.hashCode();
+      result = 31 * result + (cause != null ? cause.hashCode() : 0);
+      return result;
+    }
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b6b4af/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftHttpServlet.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftHttpServlet.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftHttpServlet.java
index 4c9a35b..7f85152 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftHttpServlet.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftHttpServlet.java
@@ -18,8 +18,8 @@
 
 package org.apache.hadoop.hbase.thrift;
 
-import static org.apache.hadoop.hbase.thrift.ThriftServerRunner.THRIFT_SPNEGO_KEYTAB_FILE_KEY;
-import static org.apache.hadoop.hbase.thrift.ThriftServerRunner.THRIFT_SPNEGO_PRINCIPAL_KEY;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_SPNEGO_KEYTAB_FILE_KEY;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_SPNEGO_PRINCIPAL_KEY;
 
 import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
@@ -58,7 +58,7 @@ public class ThriftHttpServlet extends TServlet {
   private static final Logger LOG = LoggerFactory.getLogger(ThriftHttpServlet.class.getName());
   private final transient UserGroupInformation serviceUGI;
   private final transient UserGroupInformation httpUGI;
-  private final transient ThriftServerRunner.HBaseHandler hbaseHandler;
+  private final transient HBaseServiceHandler handler;
   private final boolean doAsEnabled;
   private final boolean securityEnabled;
 
@@ -67,11 +67,11 @@ public class ThriftHttpServlet extends TServlet {
 
   public ThriftHttpServlet(TProcessor processor, TProtocolFactory protocolFactory,
       UserGroupInformation serviceUGI, Configuration conf,
-      ThriftServerRunner.HBaseHandler hbaseHandler, boolean securityEnabled, boolean doAsEnabled)
+      HBaseServiceHandler handler, boolean securityEnabled, boolean doAsEnabled)
       throws IOException {
     super(processor, protocolFactory);
     this.serviceUGI = serviceUGI;
-    this.hbaseHandler = hbaseHandler;
+    this.handler = handler;
     this.securityEnabled = securityEnabled;
     this.doAsEnabled = doAsEnabled;
 
@@ -146,7 +146,7 @@ public class ThriftHttpServlet extends TServlet {
       }
       effectiveUser = doAsUserFromQuery;
     }
-    hbaseHandler.setEffectiveUser(effectiveUser);
+    handler.setEffectiveUser(effectiveUser);
     super.doPost(request, response);
   }
 


[11/15] hbase git commit: HBASE-21630 [shell] Define ENDKEY == STOPROW (we have ENDROW)

Posted by zh...@apache.org.
HBASE-21630 [shell] Define ENDKEY == STOPROW (we have ENDROW)


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

Branch: refs/heads/HBASE-21512
Commit: 3fbdd5bbe9937bdb12984275dafced9cb6746a63
Parents: c300565
Author: Nihal Jain <ni...@gmail.com>
Authored: Wed Jan 2 01:24:08 2019 +0530
Committer: stack <st...@apache.org>
Committed: Thu Jan 3 21:58:28 2019 -0800

----------------------------------------------------------------------
 hbase-shell/src/main/ruby/hbase_constants.rb  |  2 ++
 hbase-shell/src/test/ruby/hbase/table_test.rb | 36 ++++++++++++++++++++--
 2 files changed, 35 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/3fbdd5bb/hbase-shell/src/main/ruby/hbase_constants.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase_constants.rb b/hbase-shell/src/main/ruby/hbase_constants.rb
index 4c1ad22..9871685 100644
--- a/hbase-shell/src/main/ruby/hbase_constants.rb
+++ b/hbase-shell/src/main/ruby/hbase_constants.rb
@@ -43,8 +43,10 @@ module HBaseConstants
   METADATA = org.apache.hadoop.hbase.HConstants::METADATA
   STOPROW = 'STOPROW'.freeze
   STARTROW = 'STARTROW'.freeze
+  STARTKEY = STARTROW
   ROWPREFIXFILTER = 'ROWPREFIXFILTER'.freeze
   ENDROW = STOPROW
+  ENDKEY = STOPROW
   RAW = 'RAW'.freeze
   LIMIT = 'LIMIT'.freeze
   METHOD = 'METHOD'.freeze

http://git-wip-us.apache.org/repos/asf/hbase/blob/3fbdd5bb/hbase-shell/src/test/ruby/hbase/table_test.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/ruby/hbase/table_test.rb b/hbase-shell/src/test/ruby/hbase/table_test.rb
index 5ec317a..b3343e2 100644
--- a/hbase-shell/src/test/ruby/hbase/table_test.rb
+++ b/hbase-shell/src/test/ruby/hbase/table_test.rb
@@ -476,6 +476,16 @@ module Hbase
       assert_not_nil(res['2']['x:b'])
     end
 
+    define_test "scan should support STARTKEY parameter" do
+      res = @test_table._scan_internal STARTKEY => '2'
+      assert_not_nil(res)
+      assert_kind_of(Hash, res)
+      assert_nil(res['1'])
+      assert_not_nil(res['2'])
+      assert_not_nil(res['2']['x:a'])
+      assert_not_nil(res['2']['x:b'])
+    end
+
     define_test "scan should support STOPROW parameter" do
       res = @test_table._scan_internal STOPROW => '2'
       assert_not_nil(res)
@@ -486,7 +496,27 @@ module Hbase
       assert_nil(res['2'])
     end
 
-    define_test "scan should support ROWPREFIXFILTER parameter (test 1)" do
+    define_test "scan should support ENDROW parameter" do
+      res = @test_table._scan_internal ENDROW => '2'
+      assert_not_nil(res)
+      assert_kind_of(Hash, res)
+      assert_not_nil(res['1'])
+      assert_not_nil(res['1']['x:a'])
+      assert_not_nil(res['1']['x:b'])
+      assert_nil(res['2'])
+    end
+
+    define_test "scan should support ENDKEY parameter" do
+      res = @test_table._scan_internal ENDKEY => '2'
+      assert_not_nil(res)
+      assert_kind_of(Hash, res)
+      assert_not_nil(res['1'])
+      assert_not_nil(res['1']['x:a'])
+      assert_not_nil(res['1']['x:b'])
+      assert_nil(res['2'])
+    end
+
+    define_test 'scan should support ROWPREFIXFILTER parameter (test 1)' do
       res = @test_table._scan_internal ROWPREFIXFILTER => '1'
       assert_not_nil(res)
       assert_kind_of(Hash, res)
@@ -496,7 +526,7 @@ module Hbase
       assert_nil(res['2'])
     end
 
-    define_test "scan should support ROWPREFIXFILTER parameter (test 2)" do
+    define_test 'scan should support ROWPREFIXFILTER parameter (test 2)' do
       res = @test_table._scan_internal ROWPREFIXFILTER => '2'
       assert_not_nil(res)
       assert_kind_of(Hash, res)
@@ -506,7 +536,7 @@ module Hbase
       assert_not_nil(res['2']['x:b'])
     end
 
-    define_test "scan should support LIMIT parameter" do
+    define_test 'scan should support LIMIT parameter' do
       res = @test_table._scan_internal LIMIT => 1
       assert_not_nil(res)
       assert_kind_of(Hash, res)


[06/15] hbase git commit: HBASE-17356 Add replica get support

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/db66e6cc/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTableImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTableImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTableImpl.java
index d705d7c..28db7e8 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTableImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTableImpl.java
@@ -20,9 +20,9 @@ package org.apache.hadoop.hbase.client;
 import static java.util.stream.Collectors.toList;
 import static org.apache.hadoop.hbase.client.ConnectionUtils.checkHasFamilies;
 import static org.apache.hadoop.hbase.client.ConnectionUtils.isEmptyStopRow;
+import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
 
 import com.google.protobuf.RpcChannel;
-
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -32,11 +32,11 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.function.Function;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.CompareOperator;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.RegionLocations;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.AsyncRpcRetryingCallerFactory.SingleRequestCallerBuilder;
 import org.apache.hadoop.hbase.filter.BinaryComparator;
@@ -45,9 +45,12 @@ import org.apache.hadoop.hbase.ipc.HBaseRpcController;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ReflectionUtils;
 import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
 import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter;
@@ -63,7 +66,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.CompareType
 
 /**
  * The implementation of RawAsyncTable.
- * <p>
+ * <p/>
  * The word 'Raw' means that this is a low level class. The returned {@link CompletableFuture} will
  * be finished inside the rpc framework thread, which means that the callbacks registered to the
  * {@link CompletableFuture} will also be executed inside the rpc framework thread. So users who use
@@ -74,6 +77,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.CompareType
 @InterfaceAudience.Private
 class RawAsyncTableImpl implements AsyncTable<AdvancedScanResultConsumer> {
 
+  private static final Logger LOG = LoggerFactory.getLogger(RawAsyncTableImpl.class);
+
   private final AsyncConnectionImpl conn;
 
   private final TableName tableName;
@@ -204,58 +209,126 @@ class RawAsyncTableImpl implements AsyncTable<AdvancedScanResultConsumer> {
 
   private <T> SingleRequestCallerBuilder<T> newCaller(byte[] row, long rpcTimeoutNs) {
     return conn.callerFactory.<T> single().table(tableName).row(row)
-        .rpcTimeout(rpcTimeoutNs, TimeUnit.NANOSECONDS)
-        .operationTimeout(operationTimeoutNs, TimeUnit.NANOSECONDS)
-        .pause(pauseNs, TimeUnit.NANOSECONDS).maxAttempts(maxAttempts)
-        .startLogErrorsCnt(startLogErrorsCnt);
+      .rpcTimeout(rpcTimeoutNs, TimeUnit.NANOSECONDS)
+      .operationTimeout(operationTimeoutNs, TimeUnit.NANOSECONDS)
+      .pause(pauseNs, TimeUnit.NANOSECONDS).maxAttempts(maxAttempts)
+      .startLogErrorsCnt(startLogErrorsCnt);
   }
 
   private <T> SingleRequestCallerBuilder<T> newCaller(Row row, long rpcTimeoutNs) {
     return newCaller(row.getRow(), rpcTimeoutNs);
   }
 
+  private CompletableFuture<Result> get(Get get, int replicaId, long timeoutNs) {
+    return this.<Result> newCaller(get, timeoutNs)
+      .action((controller, loc, stub) -> RawAsyncTableImpl
+        .<Get, GetRequest, GetResponse, Result> call(controller, loc, stub, get,
+          RequestConverter::buildGetRequest, (s, c, req, done) -> s.get(c, req, done),
+          (c, resp) -> ProtobufUtil.toResult(resp.getResult(), c.cellScanner())))
+      .replicaId(replicaId).call();
+  }
+
+  // Connect the two futures, if the src future is done, then mark the dst future as done. And if
+  // the dst future is done, then cancel the src future. This is used for timeline consistent read.
+  private <T> void connect(CompletableFuture<T> srcFuture, CompletableFuture<T> dstFuture) {
+    addListener(srcFuture, (r, e) -> {
+      if (e != null) {
+        dstFuture.completeExceptionally(e);
+      } else {
+        dstFuture.complete(r);
+      }
+    });
+    // The cancellation may be a dummy one as the dstFuture may be completed by this srcFuture.
+    // Notice that this is a bit tricky, as the execution chain maybe 'complete src -> complete dst
+    // -> cancel src', for now it seems to be fine, as the will use CAS to set the result first in
+    // CompletableFuture. If later this causes problems, we could use whenCompleteAsync to break the
+    // tie.
+    addListener(dstFuture, (r, e) -> srcFuture.cancel(false));
+  }
+
+  private void timelineConsistentGet(Get get, RegionLocations locs,
+      CompletableFuture<Result> future) {
+    if (future.isDone()) {
+      // do not send requests to secondary replicas if the future is done, i.e, the primary request
+      // has already been finished.
+      return;
+    }
+    for (int replicaId = 1, n = locs.size(); replicaId < n; replicaId++) {
+      CompletableFuture<Result> secondaryFuture = get(get, replicaId, readRpcTimeoutNs);
+      connect(secondaryFuture, future);
+    }
+  }
+
   @Override
   public CompletableFuture<Result> get(Get get) {
-    return this.<Result> newCaller(get, readRpcTimeoutNs)
-        .action((controller, loc, stub) -> RawAsyncTableImpl
-            .<Get, GetRequest, GetResponse, Result> call(controller, loc, stub, get,
-              RequestConverter::buildGetRequest, (s, c, req, done) -> s.get(c, req, done),
-              (c, resp) -> ProtobufUtil.toResult(resp.getResult(), c.cellScanner())))
-        .call();
+    CompletableFuture<Result> primaryFuture =
+      get(get, RegionReplicaUtil.DEFAULT_REPLICA_ID, readRpcTimeoutNs);
+    if (get.getConsistency() == Consistency.STRONG) {
+      return primaryFuture;
+    }
+    // Timeline consistent read, where we will send requests to other region replicas
+    CompletableFuture<Result> future = new CompletableFuture<>();
+    connect(primaryFuture, future);
+    long primaryCallTimeoutNs = conn.connConf.getPrimaryCallTimeoutNs();
+    long startNs = System.nanoTime();
+    addListener(conn.getLocator().getRegionLocations(tableName, get.getRow(),
+      RegionLocateType.CURRENT, false, readRpcTimeoutNs), (locs, error) -> {
+        if (error != null) {
+          LOG.warn(
+            "Failed to locate all the replicas for table={}, row='{}'," +
+              " give up timeline consistent read",
+            tableName, Bytes.toStringBinary(get.getRow()), error);
+          return;
+        }
+        if (locs.size() <= 1) {
+          LOG.warn(
+            "There are no secondary replicas for region {}," + " give up timeline consistent read",
+            locs.getDefaultRegionLocation().getRegion());
+          return;
+        }
+        long delayNs = primaryCallTimeoutNs - (System.nanoTime() - startNs);
+        if (delayNs <= 0) {
+          timelineConsistentGet(get, locs, future);
+        } else {
+          AsyncConnectionImpl.RETRY_TIMER.newTimeout(
+            timeout -> timelineConsistentGet(get, locs, future), delayNs, TimeUnit.NANOSECONDS);
+        }
+      });
+    return future;
   }
 
   @Override
   public CompletableFuture<Void> put(Put put) {
     return this.<Void> newCaller(put, writeRpcTimeoutNs)
-        .action((controller, loc, stub) -> RawAsyncTableImpl.<Put> voidMutate(controller, loc, stub,
-          put, RequestConverter::buildMutateRequest))
-        .call();
+      .action((controller, loc, stub) -> RawAsyncTableImpl.<Put> voidMutate(controller, loc, stub,
+        put, RequestConverter::buildMutateRequest))
+      .call();
   }
 
   @Override
   public CompletableFuture<Void> delete(Delete delete) {
     return this.<Void> newCaller(delete, writeRpcTimeoutNs)
-        .action((controller, loc, stub) -> RawAsyncTableImpl.<Delete> voidMutate(controller, loc,
-          stub, delete, RequestConverter::buildMutateRequest))
-        .call();
+      .action((controller, loc, stub) -> RawAsyncTableImpl.<Delete> voidMutate(controller, loc,
+        stub, delete, RequestConverter::buildMutateRequest))
+      .call();
   }
 
   @Override
   public CompletableFuture<Result> append(Append append) {
     checkHasFamilies(append);
     return this.<Result> newCaller(append, rpcTimeoutNs)
-        .action((controller, loc, stub) -> this.<Append, Result> noncedMutate(controller, loc, stub,
-          append, RequestConverter::buildMutateRequest, RawAsyncTableImpl::toResult))
-        .call();
+      .action((controller, loc, stub) -> this.<Append, Result> noncedMutate(controller, loc, stub,
+        append, RequestConverter::buildMutateRequest, RawAsyncTableImpl::toResult))
+      .call();
   }
 
   @Override
   public CompletableFuture<Result> increment(Increment increment) {
     checkHasFamilies(increment);
     return this.<Result> newCaller(increment, rpcTimeoutNs)
-        .action((controller, loc, stub) -> this.<Increment, Result> noncedMutate(controller, loc,
-          stub, increment, RequestConverter::buildMutateRequest, RawAsyncTableImpl::toResult))
-        .call();
+      .action((controller, loc, stub) -> this.<Increment, Result> noncedMutate(controller, loc,
+        stub, increment, RequestConverter::buildMutateRequest, RawAsyncTableImpl::toResult))
+      .call();
   }
 
   private final class CheckAndMutateBuilderImpl implements CheckAndMutateBuilder {
@@ -313,36 +386,36 @@ class RawAsyncTableImpl implements AsyncTable<AdvancedScanResultConsumer> {
     public CompletableFuture<Boolean> thenPut(Put put) {
       preCheck();
       return RawAsyncTableImpl.this.<Boolean> newCaller(row, rpcTimeoutNs)
-          .action((controller, loc, stub) -> RawAsyncTableImpl.<Put, Boolean> mutate(controller,
-            loc, stub, put,
-            (rn, p) -> RequestConverter.buildMutateRequest(rn, row, family, qualifier,
-              new BinaryComparator(value), CompareType.valueOf(op.name()), timeRange, p),
-            (c, r) -> r.getProcessed()))
-          .call();
+        .action((controller, loc, stub) -> RawAsyncTableImpl.<Put, Boolean> mutate(controller, loc,
+          stub, put,
+          (rn, p) -> RequestConverter.buildMutateRequest(rn, row, family, qualifier,
+            new BinaryComparator(value), CompareType.valueOf(op.name()), timeRange, p),
+          (c, r) -> r.getProcessed()))
+        .call();
     }
 
     @Override
     public CompletableFuture<Boolean> thenDelete(Delete delete) {
       preCheck();
       return RawAsyncTableImpl.this.<Boolean> newCaller(row, rpcTimeoutNs)
-          .action((controller, loc, stub) -> RawAsyncTableImpl.<Delete, Boolean> mutate(controller,
-            loc, stub, delete,
-            (rn, d) -> RequestConverter.buildMutateRequest(rn, row, family, qualifier,
-              new BinaryComparator(value), CompareType.valueOf(op.name()), timeRange, d),
-            (c, r) -> r.getProcessed()))
-          .call();
+        .action((controller, loc, stub) -> RawAsyncTableImpl.<Delete, Boolean> mutate(controller,
+          loc, stub, delete,
+          (rn, d) -> RequestConverter.buildMutateRequest(rn, row, family, qualifier,
+            new BinaryComparator(value), CompareType.valueOf(op.name()), timeRange, d),
+          (c, r) -> r.getProcessed()))
+        .call();
     }
 
     @Override
     public CompletableFuture<Boolean> thenMutate(RowMutations mutation) {
       preCheck();
       return RawAsyncTableImpl.this.<Boolean> newCaller(mutation, rpcTimeoutNs)
-          .action((controller, loc, stub) -> RawAsyncTableImpl.<Boolean> mutateRow(controller, loc,
-            stub, mutation,
-            (rn, rm) -> RequestConverter.buildMutateRequest(rn, row, family, qualifier,
-              new BinaryComparator(value), CompareType.valueOf(op.name()), timeRange, rm),
-            resp -> resp.getExists()))
-          .call();
+        .action((controller, loc, stub) -> RawAsyncTableImpl.<Boolean> mutateRow(controller, loc,
+          stub, mutation,
+          (rn, rm) -> RequestConverter.buildMutateRequest(rn, row, family, qualifier,
+            new BinaryComparator(value), CompareType.valueOf(op.name()), timeRange, rm),
+          resp -> resp.getExists()))
+        .call();
     }
   }
 
@@ -375,10 +448,10 @@ class RawAsyncTableImpl implements AsyncTable<AdvancedScanResultConsumer> {
               if (ex != null) {
                 future.completeExceptionally(ex instanceof IOException ? ex
                   : new IOException(
-                      "Failed to mutate row: " + Bytes.toStringBinary(mutation.getRow()), ex));
+                    "Failed to mutate row: " + Bytes.toStringBinary(mutation.getRow()), ex));
               } else {
                 future.complete(respConverter
-                    .apply((Result) multiResp.getResults().get(regionName).result.get(0)));
+                  .apply((Result) multiResp.getResults().get(regionName).result.get(0)));
               }
             } catch (IOException e) {
               future.completeExceptionally(e);
@@ -399,7 +472,8 @@ class RawAsyncTableImpl implements AsyncTable<AdvancedScanResultConsumer> {
           RegionAction.Builder regionMutationBuilder = RequestConverter.buildRegionAction(rn, rm);
           regionMutationBuilder.setAtomic(true);
           return MultiRequest.newBuilder().addRegionAction(regionMutationBuilder.build()).build();
-        }, resp -> null)).call();
+        }, resp -> null))
+      .call();
   }
 
   private Scan setDefaultScanConfig(Scan scan) {
@@ -416,7 +490,7 @@ class RawAsyncTableImpl implements AsyncTable<AdvancedScanResultConsumer> {
 
   public void scan(Scan scan, AdvancedScanResultConsumer consumer) {
     new AsyncClientScanner(setDefaultScanConfig(scan), consumer, tableName, conn, pauseNs,
-        maxAttempts, scanTimeoutNs, readRpcTimeoutNs, startLogErrorsCnt).start();
+      maxAttempts, scanTimeoutNs, readRpcTimeoutNs, startLogErrorsCnt).start();
   }
 
   private long resultSize2CacheSize(long maxResultSize) {
@@ -427,8 +501,8 @@ class RawAsyncTableImpl implements AsyncTable<AdvancedScanResultConsumer> {
   @Override
   public ResultScanner getScanner(Scan scan) {
     return new AsyncTableResultScanner(this, ReflectionUtils.newInstance(scan.getClass(), scan),
-        resultSize2CacheSize(
-          scan.getMaxResultSize() > 0 ? scan.getMaxResultSize() : defaultScannerMaxResultSize));
+      resultSize2CacheSize(
+        scan.getMaxResultSize() > 0 ? scan.getMaxResultSize() : defaultScannerMaxResultSize));
   }
 
   @Override
@@ -477,14 +551,14 @@ class RawAsyncTableImpl implements AsyncTable<AdvancedScanResultConsumer> {
 
   private List<CompletableFuture<Void>> voidMutate(List<? extends Row> actions) {
     return this.<Object> batch(actions, writeRpcTimeoutNs).stream()
-        .map(f -> f.<Void> thenApply(r -> null)).collect(toList());
+      .map(f -> f.<Void> thenApply(r -> null)).collect(toList());
   }
 
   private <T> List<CompletableFuture<T>> batch(List<? extends Row> actions, long rpcTimeoutNs) {
     return conn.callerFactory.batch().table(tableName).actions(actions)
-        .operationTimeout(operationTimeoutNs, TimeUnit.NANOSECONDS)
-        .rpcTimeout(rpcTimeoutNs, TimeUnit.NANOSECONDS).pause(pauseNs, TimeUnit.NANOSECONDS)
-        .maxAttempts(maxAttempts).startLogErrorsCnt(startLogErrorsCnt).call();
+      .operationTimeout(operationTimeoutNs, TimeUnit.NANOSECONDS)
+      .rpcTimeout(rpcTimeoutNs, TimeUnit.NANOSECONDS).pause(pauseNs, TimeUnit.NANOSECONDS)
+      .maxAttempts(maxAttempts).startLogErrorsCnt(startLogErrorsCnt).call();
   }
 
   @Override
@@ -515,7 +589,7 @@ class RawAsyncTableImpl implements AsyncTable<AdvancedScanResultConsumer> {
   private <S, R> CompletableFuture<R> coprocessorService(Function<RpcChannel, S> stubMaker,
       ServiceCaller<S, R> callable, RegionInfo region, byte[] row) {
     RegionCoprocessorRpcChannelImpl channel = new RegionCoprocessorRpcChannelImpl(conn, tableName,
-        region, row, rpcTimeoutNs, operationTimeoutNs);
+      region, row, rpcTimeoutNs, operationTimeoutNs);
     S stub = stubMaker.apply(channel);
     CompletableFuture<R> future = new CompletableFuture<>();
     ClientCoprocessorRpcController controller = new ClientCoprocessorRpcController();
@@ -553,10 +627,9 @@ class RawAsyncTableImpl implements AsyncTable<AdvancedScanResultConsumer> {
   }
 
   private <S, R> void onLocateComplete(Function<RpcChannel, S> stubMaker,
-      ServiceCaller<S, R> callable, CoprocessorCallback<R> callback,
-      List<HRegionLocation> locs, byte[] endKey, boolean endKeyInclusive,
-      AtomicBoolean locateFinished, AtomicInteger unfinishedRequest, HRegionLocation loc,
-      Throwable error) {
+      ServiceCaller<S, R> callable, CoprocessorCallback<R> callback, List<HRegionLocation> locs,
+      byte[] endKey, boolean endKeyInclusive, AtomicBoolean locateFinished,
+      AtomicInteger unfinishedRequest, HRegionLocation loc, Throwable error) {
     if (error != null) {
       callback.onError(error);
       return;
@@ -566,11 +639,11 @@ class RawAsyncTableImpl implements AsyncTable<AdvancedScanResultConsumer> {
     if (locateFinished(region, endKey, endKeyInclusive)) {
       locateFinished.set(true);
     } else {
-      conn.getLocator()
-          .getRegionLocation(tableName, region.getEndKey(), RegionLocateType.CURRENT,
-            operationTimeoutNs)
-          .whenComplete((l, e) -> onLocateComplete(stubMaker, callable, callback, locs, endKey,
-            endKeyInclusive, locateFinished, unfinishedRequest, l, e));
+      addListener(
+        conn.getLocator().getRegionLocation(tableName, region.getEndKey(), RegionLocateType.CURRENT,
+          operationTimeoutNs),
+        (l, e) -> onLocateComplete(stubMaker, callable, callback, locs, endKey, endKeyInclusive,
+          locateFinished, unfinishedRequest, l, e));
     }
     coprocessorService(stubMaker, callable, region, region.getStartKey()).whenComplete((r, e) -> {
       if (e != null) {
@@ -630,11 +703,10 @@ class RawAsyncTableImpl implements AsyncTable<AdvancedScanResultConsumer> {
 
     @Override
     public void execute() {
-      conn.getLocator().getRegionLocation(tableName, startKey,
-        startKeyInclusive ? RegionLocateType.CURRENT : RegionLocateType.AFTER, operationTimeoutNs)
-          .whenComplete(
-            (loc, error) -> onLocateComplete(stubMaker, callable, callback, new ArrayList<>(),
-              endKey, endKeyInclusive, new AtomicBoolean(false), new AtomicInteger(0), loc, error));
+      addListener(conn.getLocator().getRegionLocation(tableName, startKey,
+        startKeyInclusive ? RegionLocateType.CURRENT : RegionLocateType.AFTER, operationTimeoutNs),
+        (loc, error) -> onLocateComplete(stubMaker, callable, callback, new ArrayList<>(), endKey,
+          endKeyInclusive, new AtomicBoolean(false), new AtomicInteger(0), loc, error));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/db66e6cc/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FutureUtils.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FutureUtils.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FutureUtils.java
new file mode 100644
index 0000000..067e66b
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FutureUtils.java
@@ -0,0 +1,60 @@
+/**
+ * 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.util;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.function.BiConsumer;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Helper class for processing futures.
+ */
+@InterfaceAudience.Private
+public final class FutureUtils {
+
+  private static final Logger LOG = LoggerFactory.getLogger(FutureUtils.class);
+
+  private FutureUtils() {
+  }
+
+  /**
+   * This is method is used when you just want to add a listener to the given future. We will call
+   * {@link CompletableFuture#whenComplete(BiConsumer)} to register the {@code action} to the
+   * {@code future}. Ignoring the return value of a Future is considered as a bad practice as it may
+   * suppress exceptions thrown from the code that completes the future, and this method will catch
+   * all the exception thrown from the {@code action} to catch possible code bugs.
+   * <p/>
+   * And the error phone check will always report FutureReturnValueIgnored because every method in
+   * the {@link CompletableFuture} class will return a new {@link CompletableFuture}, so you always
+   * have one future that has not been checked. So we introduce this method and add a suppress
+   * warnings annotation here.
+   */
+  @SuppressWarnings("FutureReturnValueIgnored")
+  public static <T> void addListener(CompletableFuture<T> future,
+      BiConsumer<? super T, ? super Throwable> action) {
+    future.whenComplete((resp, error) -> {
+      try {
+        action.accept(resp, error);
+      } catch (Throwable t) {
+        LOG.error("Unexpected error caught when processing CompletableFuture", t);
+      }
+    });
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/db66e6cc/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RegionReplicaTestHelper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RegionReplicaTestHelper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RegionReplicaTestHelper.java
new file mode 100644
index 0000000..c14f69f
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RegionReplicaTestHelper.java
@@ -0,0 +1,161 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+import java.io.IOException;
+import java.util.Optional;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.NotServingRegionException;
+import org.apache.hadoop.hbase.RegionLocations;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
+import org.apache.hadoop.hbase.util.Bytes;
+
+final class RegionReplicaTestHelper {
+
+  private RegionReplicaTestHelper() {
+  }
+
+  // waits for all replicas to have region location
+  static void waitUntilAllMetaReplicasHavingRegionLocation(AsyncRegistry registry,
+      int regionReplication) throws IOException {
+    TestZKAsyncRegistry.TEST_UTIL.waitFor(
+      TestZKAsyncRegistry.TEST_UTIL.getConfiguration()
+        .getLong("hbase.client.sync.wait.timeout.msec", 60000),
+      200, true, new ExplainingPredicate<IOException>() {
+        @Override
+        public String explainFailure() throws IOException {
+          return "Not all meta replicas get assigned";
+        }
+
+        @Override
+        public boolean evaluate() throws IOException {
+          try {
+            RegionLocations locs = registry.getMetaRegionLocation().get();
+            if (locs.size() < regionReplication) {
+              return false;
+            }
+            for (int i = 0; i < regionReplication; i++) {
+              if (locs.getRegionLocation(i) == null) {
+                return false;
+              }
+            }
+            return true;
+          } catch (Exception e) {
+            TestZKAsyncRegistry.LOG.warn("Failed to get meta region locations", e);
+            return false;
+          }
+        }
+      });
+  }
+
+  static Optional<ServerName> getRSCarryingReplica(HBaseTestingUtility util, TableName tableName,
+      int replicaId) {
+    return util.getHBaseCluster().getRegionServerThreads().stream().map(t -> t.getRegionServer())
+      .filter(rs -> rs.getRegions(tableName).stream()
+        .anyMatch(r -> r.getRegionInfo().getReplicaId() == replicaId))
+      .findAny().map(rs -> rs.getServerName());
+  }
+
+  /**
+   * Return the new location.
+   */
+  static ServerName moveRegion(HBaseTestingUtility util, HRegionLocation currentLoc)
+      throws Exception {
+    ServerName serverName = currentLoc.getServerName();
+    RegionInfo regionInfo = currentLoc.getRegion();
+    TableName tableName = regionInfo.getTable();
+    int replicaId = regionInfo.getReplicaId();
+    ServerName newServerName = util.getHBaseCluster().getRegionServerThreads().stream()
+      .map(t -> t.getRegionServer().getServerName()).filter(sn -> !sn.equals(serverName)).findAny()
+      .get();
+    util.getAdmin().move(regionInfo.getEncodedNameAsBytes(),
+      Bytes.toBytes(newServerName.getServerName()));
+    util.waitFor(30000, new ExplainingPredicate<Exception>() {
+
+      @Override
+      public boolean evaluate() throws Exception {
+        Optional<ServerName> newServerName = getRSCarryingReplica(util, tableName, replicaId);
+        return newServerName.isPresent() && !newServerName.get().equals(serverName);
+      }
+
+      @Override
+      public String explainFailure() throws Exception {
+        return regionInfo.getRegionNameAsString() + " is still on " + serverName;
+      }
+    });
+    return newServerName;
+  }
+
+  interface Locator {
+    RegionLocations getRegionLocations(TableName tableName, int replicaId, boolean reload)
+        throws Exception;
+
+    void updateCachedLocationOnError(HRegionLocation loc, Throwable error) throws Exception;
+  }
+
+  static void testLocator(HBaseTestingUtility util, TableName tableName, Locator locator)
+      throws Exception {
+    RegionLocations locs =
+      locator.getRegionLocations(tableName, RegionReplicaUtil.DEFAULT_REPLICA_ID, false);
+    assertEquals(3, locs.size());
+    for (int i = 0; i < 3; i++) {
+      HRegionLocation loc = locs.getRegionLocation(i);
+      assertNotNull(loc);
+      ServerName serverName = getRSCarryingReplica(util, tableName, i).get();
+      assertEquals(serverName, loc.getServerName());
+    }
+    ServerName newServerName = moveRegion(util, locs.getDefaultRegionLocation());
+    // The cached location should not be changed
+    assertEquals(locs.getDefaultRegionLocation().getServerName(),
+      locator.getRegionLocations(tableName, RegionReplicaUtil.DEFAULT_REPLICA_ID, false)
+        .getDefaultRegionLocation().getServerName());
+    // should get the new location when reload = true
+    assertEquals(newServerName,
+      locator.getRegionLocations(tableName, RegionReplicaUtil.DEFAULT_REPLICA_ID, true)
+        .getDefaultRegionLocation().getServerName());
+    // the cached location should be replaced
+    assertEquals(newServerName,
+      locator.getRegionLocations(tableName, RegionReplicaUtil.DEFAULT_REPLICA_ID, false)
+        .getDefaultRegionLocation().getServerName());
+
+    ServerName newServerName1 = moveRegion(util, locs.getRegionLocation(1));
+    ServerName newServerName2 = moveRegion(util, locs.getRegionLocation(2));
+
+    // The cached location should not be change
+    assertEquals(locs.getRegionLocation(1).getServerName(),
+      locator.getRegionLocations(tableName, 1, false).getRegionLocation(1).getServerName());
+    // clear the cached location for replica 1
+    locator.updateCachedLocationOnError(locs.getRegionLocation(1), new NotServingRegionException());
+    // the cached location for replica 2 should not be changed
+    assertEquals(locs.getRegionLocation(2).getServerName(),
+      locator.getRegionLocations(tableName, 2, false).getRegionLocation(2).getServerName());
+    // should get the new location as we have cleared the old location
+    assertEquals(newServerName1,
+      locator.getRegionLocations(tableName, 1, false).getRegionLocation(1).getServerName());
+    // as we will get the new location for replica 2 at once, we should also get the new location
+    // for replica 2
+    assertEquals(newServerName2,
+      locator.getRegionLocations(tableName, 2, false).getRegionLocation(2).getServerName());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/db66e6cc/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocator.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocator.java
index 7c08d6d..df1fe08 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocator.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocator.java
@@ -17,20 +17,19 @@
  */
 package org.apache.hadoop.hbase.client;
 
-import static org.junit.Assert.assertEquals;
+import static org.apache.hadoop.hbase.client.RegionReplicaTestHelper.testLocator;
 
-import java.util.Optional;
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.RegionLocations;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
+import org.apache.hadoop.hbase.client.RegionReplicaTestHelper.Locator;
 import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
@@ -42,7 +41,7 @@ public class TestAsyncMetaRegionLocator {
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestAsyncMetaRegionLocator.class);
+    HBaseClassTestRule.forClass(TestAsyncMetaRegionLocator.class);
 
   private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 
@@ -53,10 +52,11 @@ public class TestAsyncMetaRegionLocator {
   @BeforeClass
   public static void setUp() throws Exception {
     TEST_UTIL.getConfiguration().set(BaseLoadBalancer.TABLES_ON_MASTER, "none");
+    TEST_UTIL.getConfiguration().setInt(HConstants.META_REPLICAS_NUM, 3);
     TEST_UTIL.startMiniCluster(3);
-    TEST_UTIL.waitUntilAllSystemRegionsAssigned();
-    TEST_UTIL.getAdmin().setBalancerRunning(false, true);
     REGISTRY = AsyncRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
+    RegionReplicaTestHelper.waitUntilAllMetaReplicasHavingRegionLocation(REGISTRY, 3);
+    TEST_UTIL.getAdmin().balancerSwitch(false, true);
     LOCATOR = new AsyncMetaRegionLocator(REGISTRY);
   }
 
@@ -66,42 +66,21 @@ public class TestAsyncMetaRegionLocator {
     TEST_UTIL.shutdownMiniCluster();
   }
 
-  private Optional<ServerName> getRSCarryingMeta() {
-    return TEST_UTIL.getHBaseCluster().getRegionServerThreads().stream()
-        .map(t -> t.getRegionServer())
-        .filter(rs -> !rs.getRegions(TableName.META_TABLE_NAME).isEmpty()).findAny()
-        .map(rs -> rs.getServerName());
-  }
-
   @Test
-  public void testReload() throws Exception {
-    ServerName serverName = getRSCarryingMeta().get();
-    assertEquals(serverName, LOCATOR.getRegionLocation(false).get().getServerName());
-
-    ServerName newServerName = TEST_UTIL.getHBaseCluster().getRegionServerThreads().stream()
-        .map(t -> t.getRegionServer().getServerName()).filter(sn -> !sn.equals(serverName))
-        .findAny().get();
-    TEST_UTIL.getAdmin().move(HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes(),
-      Bytes.toBytes(newServerName.getServerName()));
-    TEST_UTIL.waitFor(30000, new ExplainingPredicate<Exception>() {
+  public void test() throws Exception {
+    testLocator(TEST_UTIL, TableName.META_TABLE_NAME, new Locator() {
 
       @Override
-      public boolean evaluate() throws Exception {
-        Optional<ServerName> newServerName = getRSCarryingMeta();
-        return newServerName.isPresent() && !newServerName.get().equals(serverName);
+      public void updateCachedLocationOnError(HRegionLocation loc, Throwable error)
+          throws Exception {
+        LOCATOR.updateCachedLocationOnError(loc, error);
       }
 
       @Override
-      public String explainFailure() throws Exception {
-        return HRegionInfo.FIRST_META_REGIONINFO.getRegionNameAsString() + " is still on " +
-            serverName;
+      public RegionLocations getRegionLocations(TableName tableName, int replicaId, boolean reload)
+          throws Exception {
+        return LOCATOR.getRegionLocations(replicaId, reload).get();
       }
     });
-    // The cached location will not change
-    assertEquals(serverName, LOCATOR.getRegionLocation(false).get().getServerName());
-    // should get the new location when reload = true
-    assertEquals(newServerName, LOCATOR.getRegionLocation(true).get().getServerName());
-    // the cached location should be replaced
-    assertEquals(newServerName, LOCATOR.getRegionLocation(false).get().getServerName());
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/db66e6cc/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java
index 38dc78d..eeaf99f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.client;
 import static java.util.stream.Collectors.toList;
 import static org.apache.hadoop.hbase.HConstants.EMPTY_END_ROW;
 import static org.apache.hadoop.hbase.HConstants.EMPTY_START_ROW;
+import static org.apache.hadoop.hbase.client.RegionReplicaTestHelper.testLocator;
 import static org.hamcrest.CoreMatchers.instanceOf;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
@@ -38,10 +39,12 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.NotServingRegionException;
+import org.apache.hadoop.hbase.RegionLocations;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
+import org.apache.hadoop.hbase.client.RegionReplicaTestHelper.Locator;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -58,7 +61,7 @@ public class TestAsyncNonMetaRegionLocator {
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestAsyncNonMetaRegionLocator.class);
+    HBaseClassTestRule.forClass(TestAsyncNonMetaRegionLocator.class);
 
   private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 
@@ -78,7 +81,7 @@ public class TestAsyncNonMetaRegionLocator {
     TEST_UTIL.getAdmin().balancerSwitch(false, true);
     AsyncRegistry registry = AsyncRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
     CONN = new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), registry,
-        registry.getClusterId().get(), User.getCurrent());
+      registry.getClusterId().get(), User.getCurrent());
     LOCATOR = new AsyncNonMetaRegionLocator(CONN);
     SPLIT_KEYS = new byte[8][];
     for (int i = 111; i < 999; i += 111) {
@@ -109,11 +112,18 @@ public class TestAsyncNonMetaRegionLocator {
     TEST_UTIL.waitTableAvailable(TABLE_NAME);
   }
 
+  private CompletableFuture<HRegionLocation> getDefaultRegionLocation(TableName tableName,
+      byte[] row, RegionLocateType locateType, boolean reload) {
+    return LOCATOR
+      .getRegionLocations(tableName, row, RegionReplicaUtil.DEFAULT_REPLICA_ID, locateType, reload)
+      .thenApply(RegionLocations::getDefaultRegionLocation);
+  }
+
   @Test
   public void testNoTable() throws InterruptedException {
     for (RegionLocateType locateType : RegionLocateType.values()) {
       try {
-        LOCATOR.getRegionLocation(TABLE_NAME, EMPTY_START_ROW, locateType, false).get();
+        getDefaultRegionLocation(TABLE_NAME, EMPTY_START_ROW, locateType, false).get();
       } catch (ExecutionException e) {
         assertThat(e.getCause(), instanceOf(TableNotFoundException.class));
       }
@@ -126,7 +136,7 @@ public class TestAsyncNonMetaRegionLocator {
     TEST_UTIL.getAdmin().disableTable(TABLE_NAME);
     for (RegionLocateType locateType : RegionLocateType.values()) {
       try {
-        LOCATOR.getRegionLocation(TABLE_NAME, EMPTY_START_ROW, locateType, false).get();
+        getDefaultRegionLocation(TABLE_NAME, EMPTY_START_ROW, locateType, false).get();
       } catch (ExecutionException e) {
         assertThat(e.getCause(), instanceOf(TableNotFoundException.class));
       }
@@ -148,13 +158,13 @@ public class TestAsyncNonMetaRegionLocator {
     ServerName serverName = TEST_UTIL.getRSForFirstRegionInTable(TABLE_NAME).getServerName();
     for (RegionLocateType locateType : RegionLocateType.values()) {
       assertLocEquals(EMPTY_START_ROW, EMPTY_END_ROW, serverName,
-        LOCATOR.getRegionLocation(TABLE_NAME, EMPTY_START_ROW, locateType, false).get());
+        getDefaultRegionLocation(TABLE_NAME, EMPTY_START_ROW, locateType, false).get());
     }
     byte[] randKey = new byte[ThreadLocalRandom.current().nextInt(128)];
     ThreadLocalRandom.current().nextBytes(randKey);
     for (RegionLocateType locateType : RegionLocateType.values()) {
       assertLocEquals(EMPTY_START_ROW, EMPTY_END_ROW, serverName,
-        LOCATOR.getRegionLocation(TABLE_NAME, randKey, locateType, false).get());
+        getDefaultRegionLocation(TABLE_NAME, randKey, locateType, false).get());
     }
   }
 
@@ -179,12 +189,12 @@ public class TestAsyncNonMetaRegionLocator {
   private ServerName[] getLocations(byte[][] startKeys) {
     ServerName[] serverNames = new ServerName[startKeys.length];
     TEST_UTIL.getHBaseCluster().getRegionServerThreads().stream().map(t -> t.getRegionServer())
-        .forEach(rs -> {
-          rs.getRegions(TABLE_NAME).forEach(r -> {
-            serverNames[Arrays.binarySearch(startKeys, r.getRegionInfo().getStartKey(),
-              Bytes::compareTo)] = rs.getServerName();
-          });
+      .forEach(rs -> {
+        rs.getRegions(TABLE_NAME).forEach(r -> {
+          serverNames[Arrays.binarySearch(startKeys, r.getRegionInfo().getStartKey(),
+            Bytes::compareTo)] = rs.getServerName();
         });
+      });
     return serverNames;
   }
 
@@ -196,8 +206,9 @@ public class TestAsyncNonMetaRegionLocator {
     IntStream.range(0, 2).forEach(n -> IntStream.range(0, startKeys.length).forEach(i -> {
       try {
         assertLocEquals(startKeys[i], i == startKeys.length - 1 ? EMPTY_END_ROW : startKeys[i + 1],
-          serverNames[i], LOCATOR
-              .getRegionLocation(TABLE_NAME, startKeys[i], RegionLocateType.CURRENT, false).get());
+          serverNames[i],
+          getDefaultRegionLocation(TABLE_NAME, startKeys[i], RegionLocateType.CURRENT, false)
+            .get());
       } catch (InterruptedException | ExecutionException e) {
         throw new RuntimeException(e);
       }
@@ -208,7 +219,7 @@ public class TestAsyncNonMetaRegionLocator {
       try {
         assertLocEquals(startKeys[i], i == startKeys.length - 1 ? EMPTY_END_ROW : startKeys[i + 1],
           serverNames[i],
-          LOCATOR.getRegionLocation(TABLE_NAME, startKeys[i], RegionLocateType.AFTER, false).get());
+          getDefaultRegionLocation(TABLE_NAME, startKeys[i], RegionLocateType.AFTER, false).get());
       } catch (InterruptedException | ExecutionException e) {
         throw new RuntimeException(e);
       }
@@ -220,8 +231,7 @@ public class TestAsyncNonMetaRegionLocator {
       n -> IntStream.range(0, endKeys.length).map(i -> endKeys.length - 1 - i).forEach(i -> {
         try {
           assertLocEquals(i == 0 ? EMPTY_START_ROW : endKeys[i - 1], endKeys[i], serverNames[i],
-            LOCATOR.getRegionLocation(TABLE_NAME, endKeys[i], RegionLocateType.BEFORE, false)
-                .get());
+            getDefaultRegionLocation(TABLE_NAME, endKeys[i], RegionLocateType.BEFORE, false).get());
         } catch (InterruptedException | ExecutionException e) {
           throw new RuntimeException(e);
         }
@@ -232,29 +242,29 @@ public class TestAsyncNonMetaRegionLocator {
   public void testRegionMove() throws IOException, InterruptedException, ExecutionException {
     createSingleRegionTable();
     ServerName serverName = TEST_UTIL.getRSForFirstRegionInTable(TABLE_NAME).getServerName();
-    HRegionLocation loc = LOCATOR
-        .getRegionLocation(TABLE_NAME, EMPTY_START_ROW, RegionLocateType.CURRENT, false).get();
+    HRegionLocation loc =
+      getDefaultRegionLocation(TABLE_NAME, EMPTY_START_ROW, RegionLocateType.CURRENT, false).get();
     assertLocEquals(EMPTY_START_ROW, EMPTY_END_ROW, serverName, loc);
     ServerName newServerName = TEST_UTIL.getHBaseCluster().getRegionServerThreads().stream()
-        .map(t -> t.getRegionServer().getServerName()).filter(sn -> !sn.equals(serverName))
-        .findAny().get();
+      .map(t -> t.getRegionServer().getServerName()).filter(sn -> !sn.equals(serverName)).findAny()
+      .get();
 
     TEST_UTIL.getAdmin().move(Bytes.toBytes(loc.getRegion().getEncodedName()),
       Bytes.toBytes(newServerName.getServerName()));
     while (!TEST_UTIL.getRSForFirstRegionInTable(TABLE_NAME).getServerName()
-        .equals(newServerName)) {
+      .equals(newServerName)) {
       Thread.sleep(100);
     }
     // Should be same as it is in cache
-    assertSame(loc, LOCATOR
-        .getRegionLocation(TABLE_NAME, EMPTY_START_ROW, RegionLocateType.CURRENT, false).get());
-    LOCATOR.updateCachedLocation(loc, null);
+    assertSame(loc,
+      getDefaultRegionLocation(TABLE_NAME, EMPTY_START_ROW, RegionLocateType.CURRENT, false).get());
+    LOCATOR.updateCachedLocationOnError(loc, null);
     // null error will not trigger a cache cleanup
-    assertSame(loc, LOCATOR
-        .getRegionLocation(TABLE_NAME, EMPTY_START_ROW, RegionLocateType.CURRENT, false).get());
-    LOCATOR.updateCachedLocation(loc, new NotServingRegionException());
-    assertLocEquals(EMPTY_START_ROW, EMPTY_END_ROW, newServerName, LOCATOR
-        .getRegionLocation(TABLE_NAME, EMPTY_START_ROW, RegionLocateType.CURRENT, false).get());
+    assertSame(loc,
+      getDefaultRegionLocation(TABLE_NAME, EMPTY_START_ROW, RegionLocateType.CURRENT, false).get());
+    LOCATOR.updateCachedLocationOnError(loc, new NotServingRegionException());
+    assertLocEquals(EMPTY_START_ROW, EMPTY_END_ROW, newServerName,
+      getDefaultRegionLocation(TABLE_NAME, EMPTY_START_ROW, RegionLocateType.CURRENT, false).get());
   }
 
   // usually locate after will return the same result, so we add a test to make it return different
@@ -266,21 +276,21 @@ public class TestAsyncNonMetaRegionLocator {
     TEST_UTIL.createTable(TABLE_NAME, FAMILY, new byte[][] { splitKey });
     TEST_UTIL.waitTableAvailable(TABLE_NAME);
     HRegionLocation currentLoc =
-        LOCATOR.getRegionLocation(TABLE_NAME, row, RegionLocateType.CURRENT, false).get();
+      getDefaultRegionLocation(TABLE_NAME, row, RegionLocateType.CURRENT, false).get();
     ServerName currentServerName = TEST_UTIL.getRSForFirstRegionInTable(TABLE_NAME).getServerName();
     assertLocEquals(EMPTY_START_ROW, splitKey, currentServerName, currentLoc);
 
     HRegionLocation afterLoc =
-        LOCATOR.getRegionLocation(TABLE_NAME, row, RegionLocateType.AFTER, false).get();
+      getDefaultRegionLocation(TABLE_NAME, row, RegionLocateType.AFTER, false).get();
     ServerName afterServerName =
-        TEST_UTIL.getHBaseCluster().getRegionServerThreads().stream().map(t -> t.getRegionServer())
-            .filter(rs -> rs.getRegions(TABLE_NAME).stream()
-                .anyMatch(r -> Bytes.equals(splitKey, r.getRegionInfo().getStartKey())))
-            .findAny().get().getServerName();
+      TEST_UTIL.getHBaseCluster().getRegionServerThreads().stream().map(t -> t.getRegionServer())
+        .filter(rs -> rs.getRegions(TABLE_NAME).stream()
+          .anyMatch(r -> Bytes.equals(splitKey, r.getRegionInfo().getStartKey())))
+        .findAny().get().getServerName();
     assertLocEquals(splitKey, EMPTY_END_ROW, afterServerName, afterLoc);
 
     assertSame(afterLoc,
-      LOCATOR.getRegionLocation(TABLE_NAME, row, RegionLocateType.AFTER, false).get());
+      getDefaultRegionLocation(TABLE_NAME, row, RegionLocateType.AFTER, false).get());
   }
 
   // For HBASE-17402
@@ -292,9 +302,9 @@ public class TestAsyncNonMetaRegionLocator {
     ServerName[] serverNames = getLocations(startKeys);
     for (int i = 0; i < 100; i++) {
       LOCATOR.clearCache(TABLE_NAME);
-      List<CompletableFuture<HRegionLocation>> futures = IntStream.range(0, 1000)
-          .mapToObj(n -> String.format("%03d", n)).map(s -> Bytes.toBytes(s))
-          .map(r -> LOCATOR.getRegionLocation(TABLE_NAME, r, RegionLocateType.CURRENT, false))
+      List<CompletableFuture<HRegionLocation>> futures =
+        IntStream.range(0, 1000).mapToObj(n -> String.format("%03d", n)).map(s -> Bytes.toBytes(s))
+          .map(r -> getDefaultRegionLocation(TABLE_NAME, r, RegionLocateType.CURRENT, false))
           .collect(toList());
       for (int j = 0; j < 1000; j++) {
         int index = Math.min(8, j / 111);
@@ -309,11 +319,11 @@ public class TestAsyncNonMetaRegionLocator {
     ServerName serverName = TEST_UTIL.getRSForFirstRegionInTable(TABLE_NAME).getServerName();
     for (RegionLocateType locateType : RegionLocateType.values()) {
       assertLocEquals(EMPTY_START_ROW, EMPTY_END_ROW, serverName,
-        LOCATOR.getRegionLocation(TABLE_NAME, EMPTY_START_ROW, locateType, false).get());
+        getDefaultRegionLocation(TABLE_NAME, EMPTY_START_ROW, locateType, false).get());
     }
     ServerName newServerName = TEST_UTIL.getHBaseCluster().getRegionServerThreads().stream()
-        .map(t -> t.getRegionServer().getServerName()).filter(sn -> !sn.equals(serverName))
-        .findAny().get();
+      .map(t -> t.getRegionServer().getServerName()).filter(sn -> !sn.equals(serverName)).findAny()
+      .get();
     Admin admin = TEST_UTIL.getAdmin();
     RegionInfo region = admin.getRegions(TABLE_NAME).stream().findAny().get();
     admin.move(region.getEncodedNameAsBytes(), Bytes.toBytes(newServerName.getServerName()));
@@ -334,15 +344,15 @@ public class TestAsyncNonMetaRegionLocator {
     // The cached location will not change
     for (RegionLocateType locateType : RegionLocateType.values()) {
       assertLocEquals(EMPTY_START_ROW, EMPTY_END_ROW, serverName,
-        LOCATOR.getRegionLocation(TABLE_NAME, EMPTY_START_ROW, locateType, false).get());
+        getDefaultRegionLocation(TABLE_NAME, EMPTY_START_ROW, locateType, false).get());
     }
     // should get the new location when reload = true
     assertLocEquals(EMPTY_START_ROW, EMPTY_END_ROW, newServerName,
-      LOCATOR.getRegionLocation(TABLE_NAME, EMPTY_START_ROW, RegionLocateType.CURRENT, true).get());
+      getDefaultRegionLocation(TABLE_NAME, EMPTY_START_ROW, RegionLocateType.CURRENT, true).get());
     // the cached location should be replaced
     for (RegionLocateType locateType : RegionLocateType.values()) {
       assertLocEquals(EMPTY_START_ROW, EMPTY_END_ROW, newServerName,
-        LOCATOR.getRegionLocation(TABLE_NAME, EMPTY_START_ROW, locateType, false).get());
+        getDefaultRegionLocation(TABLE_NAME, EMPTY_START_ROW, locateType, false).get());
     }
   }
 
@@ -351,10 +361,32 @@ public class TestAsyncNonMetaRegionLocator {
   public void testLocateBeforeLastRegion()
       throws IOException, InterruptedException, ExecutionException {
     createMultiRegionTable();
-    LOCATOR.getRegionLocation(TABLE_NAME, SPLIT_KEYS[0], RegionLocateType.CURRENT, false).join();
+    getDefaultRegionLocation(TABLE_NAME, SPLIT_KEYS[0], RegionLocateType.CURRENT, false).join();
     HRegionLocation loc =
-      LOCATOR.getRegionLocation(TABLE_NAME, EMPTY_END_ROW, RegionLocateType.BEFORE, false).get();
+      getDefaultRegionLocation(TABLE_NAME, EMPTY_END_ROW, RegionLocateType.BEFORE, false).get();
     // should locate to the last region
     assertArrayEquals(loc.getRegion().getEndKey(), EMPTY_END_ROW);
   }
+
+  @Test
+  public void testRegionReplicas() throws Exception {
+    TEST_UTIL.getAdmin().createTable(TableDescriptorBuilder.newBuilder(TABLE_NAME)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY)).setRegionReplication(3).build());
+    TEST_UTIL.waitUntilAllRegionsAssigned(TABLE_NAME);
+    testLocator(TEST_UTIL, TABLE_NAME, new Locator() {
+
+      @Override
+      public void updateCachedLocationOnError(HRegionLocation loc, Throwable error)
+          throws Exception {
+        LOCATOR.updateCachedLocationOnError(loc, error);
+      }
+
+      @Override
+      public RegionLocations getRegionLocations(TableName tableName, int replicaId, boolean reload)
+          throws Exception {
+        return LOCATOR.getRegionLocations(tableName, EMPTY_START_ROW, replicaId,
+          RegionLocateType.CURRENT, reload).get();
+      }
+    });
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/db66e6cc/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java
index c6624e7..8cdb4a9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.RegionLocations;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
@@ -59,7 +60,7 @@ public class TestAsyncNonMetaRegionLocatorConcurrenyLimit {
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestAsyncNonMetaRegionLocatorConcurrenyLimit.class);
+    HBaseClassTestRule.forClass(TestAsyncNonMetaRegionLocatorConcurrenyLimit.class);
 
   private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 
@@ -124,10 +125,10 @@ public class TestAsyncNonMetaRegionLocatorConcurrenyLimit {
     TEST_UTIL.getAdmin().balancerSwitch(false, true);
     AsyncRegistry registry = AsyncRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
     CONN = new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), registry,
-        registry.getClusterId().get(), User.getCurrent());
+      registry.getClusterId().get(), User.getCurrent());
     LOCATOR = new AsyncNonMetaRegionLocator(CONN);
     SPLIT_KEYS = IntStream.range(1, 256).mapToObj(i -> Bytes.toBytes(String.format("%02x", i)))
-        .toArray(byte[][]::new);
+      .toArray(byte[][]::new);
     TEST_UTIL.createTable(TABLE_NAME, FAMILY, SPLIT_KEYS);
     TEST_UTIL.waitTableAvailable(TABLE_NAME);
   }
@@ -138,11 +139,11 @@ public class TestAsyncNonMetaRegionLocatorConcurrenyLimit {
     TEST_UTIL.shutdownMiniCluster();
   }
 
-  private void assertLocs(List<CompletableFuture<HRegionLocation>> futures)
+  private void assertLocs(List<CompletableFuture<RegionLocations>> futures)
       throws InterruptedException, ExecutionException {
     assertEquals(256, futures.size());
     for (int i = 0; i < futures.size(); i++) {
-      HRegionLocation loc = futures.get(i).get();
+      HRegionLocation loc = futures.get(i).get().getDefaultRegionLocation();
       if (i == 0) {
         assertTrue(isEmptyStartRow(loc.getRegion().getStartKey()));
       } else {
@@ -158,10 +159,11 @@ public class TestAsyncNonMetaRegionLocatorConcurrenyLimit {
 
   @Test
   public void test() throws InterruptedException, ExecutionException {
-    List<CompletableFuture<HRegionLocation>> futures =
-        IntStream.range(0, 256).mapToObj(i -> Bytes.toBytes(String.format("%02x", i)))
-            .map(r -> LOCATOR.getRegionLocation(TABLE_NAME, r, RegionLocateType.CURRENT, false))
-            .collect(toList());
+    List<CompletableFuture<RegionLocations>> futures =
+      IntStream.range(0, 256).mapToObj(i -> Bytes.toBytes(String.format("%02x", i)))
+        .map(r -> LOCATOR.getRegionLocations(TABLE_NAME, r, RegionReplicaUtil.DEFAULT_REPLICA_ID,
+          RegionLocateType.CURRENT, false))
+        .collect(toList());
     assertLocs(futures);
     assertTrue("max allowed is " + MAX_ALLOWED + " but actual is " + MAX_CONCURRENCY.get(),
       MAX_CONCURRENCY.get() <= MAX_ALLOWED);

http://git-wip-us.apache.org/repos/asf/hbase/blob/db66e6cc/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java
index a6c2efb..7d8956b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java
@@ -49,7 +49,7 @@ public class TestAsyncSingleRequestRpcRetryingCaller {
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestAsyncSingleRequestRpcRetryingCaller.class);
+    HBaseClassTestRule.forClass(TestAsyncSingleRequestRpcRetryingCaller.class);
 
   private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 
@@ -73,7 +73,7 @@ public class TestAsyncSingleRequestRpcRetryingCaller {
     TEST_UTIL.waitTableAvailable(TABLE_NAME);
     AsyncRegistry registry = AsyncRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
     CONN = new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), registry,
-        registry.getClusterId().get(), User.getCurrent());
+      registry.getClusterId().get(), User.getCurrent());
   }
 
   @AfterClass
@@ -89,8 +89,8 @@ public class TestAsyncSingleRequestRpcRetryingCaller {
     int index = TEST_UTIL.getHBaseCluster().getServerWith(loc.getRegion().getRegionName());
     TEST_UTIL.getAdmin().move(loc.getRegion().getEncodedNameAsBytes(), Bytes.toBytes(
       TEST_UTIL.getHBaseCluster().getRegionServer(1 - index).getServerName().getServerName()));
-    AsyncTable<?> table = CONN.getTableBuilder(TABLE_NAME)
-        .setRetryPause(100, TimeUnit.MILLISECONDS).setMaxRetries(30).build();
+    AsyncTable<?> table = CONN.getTableBuilder(TABLE_NAME).setRetryPause(100, TimeUnit.MILLISECONDS)
+      .setMaxRetries(30).build();
     table.put(new Put(ROW).addColumn(FAMILY, QUALIFIER, VALUE)).get();
 
     // move back
@@ -110,8 +110,8 @@ public class TestAsyncSingleRequestRpcRetryingCaller {
   public void testMaxRetries() throws IOException, InterruptedException {
     try {
       CONN.callerFactory.single().table(TABLE_NAME).row(ROW).operationTimeout(1, TimeUnit.DAYS)
-          .maxAttempts(3).pause(10, TimeUnit.MILLISECONDS)
-          .action((controller, loc, stub) -> failedFuture()).call().get();
+        .maxAttempts(3).pause(10, TimeUnit.MILLISECONDS)
+        .action((controller, loc, stub) -> failedFuture()).call().get();
       fail();
     } catch (ExecutionException e) {
       assertThat(e.getCause(), instanceOf(RetriesExhaustedException.class));
@@ -123,8 +123,8 @@ public class TestAsyncSingleRequestRpcRetryingCaller {
     long startNs = System.nanoTime();
     try {
       CONN.callerFactory.single().table(TABLE_NAME).row(ROW).operationTimeout(1, TimeUnit.SECONDS)
-          .pause(100, TimeUnit.MILLISECONDS).maxAttempts(Integer.MAX_VALUE)
-          .action((controller, loc, stub) -> failedFuture()).call().get();
+        .pause(100, TimeUnit.MILLISECONDS).maxAttempts(Integer.MAX_VALUE)
+        .action((controller, loc, stub) -> failedFuture()).call().get();
       fail();
     } catch (ExecutionException e) {
       e.printStackTrace();
@@ -141,30 +141,30 @@ public class TestAsyncSingleRequestRpcRetryingCaller {
     AtomicInteger count = new AtomicInteger(0);
     HRegionLocation loc = CONN.getRegionLocator(TABLE_NAME).getRegionLocation(ROW).get();
     AsyncRegionLocator mockedLocator =
-        new AsyncRegionLocator(CONN, AsyncConnectionImpl.RETRY_TIMER) {
-          @Override
-          CompletableFuture<HRegionLocation> getRegionLocation(TableName tableName, byte[] row,
-              RegionLocateType locateType, long timeoutNs) {
-            if (tableName.equals(TABLE_NAME)) {
-              CompletableFuture<HRegionLocation> future = new CompletableFuture<>();
-              if (count.getAndIncrement() == 0) {
-                errorTriggered.set(true);
-                future.completeExceptionally(new RuntimeException("Inject error!"));
-              } else {
-                future.complete(loc);
-              }
-              return future;
+      new AsyncRegionLocator(CONN, AsyncConnectionImpl.RETRY_TIMER) {
+        @Override
+        CompletableFuture<HRegionLocation> getRegionLocation(TableName tableName, byte[] row,
+            int replicaId, RegionLocateType locateType, long timeoutNs) {
+          if (tableName.equals(TABLE_NAME)) {
+            CompletableFuture<HRegionLocation> future = new CompletableFuture<>();
+            if (count.getAndIncrement() == 0) {
+              errorTriggered.set(true);
+              future.completeExceptionally(new RuntimeException("Inject error!"));
             } else {
-              return super.getRegionLocation(tableName, row, locateType, timeoutNs);
+              future.complete(loc);
             }
+            return future;
+          } else {
+            return super.getRegionLocation(tableName, row, replicaId, locateType, timeoutNs);
           }
+        }
 
-          @Override
-          void updateCachedLocation(HRegionLocation loc, Throwable exception) {
-          }
-        };
+        @Override
+        void updateCachedLocationOnError(HRegionLocation loc, Throwable exception) {
+        }
+      };
     try (AsyncConnectionImpl mockedConn = new AsyncConnectionImpl(CONN.getConfiguration(),
-        CONN.registry, CONN.registry.getClusterId().get(), User.getCurrent()) {
+      CONN.registry, CONN.registry.getClusterId().get(), User.getCurrent()) {
 
       @Override
       AsyncRegionLocator getLocator() {
@@ -172,7 +172,7 @@ public class TestAsyncSingleRequestRpcRetryingCaller {
       }
     }) {
       AsyncTable<?> table = mockedConn.getTableBuilder(TABLE_NAME)
-          .setRetryPause(100, TimeUnit.MILLISECONDS).setMaxRetries(5).build();
+        .setRetryPause(100, TimeUnit.MILLISECONDS).setMaxRetries(5).build();
       table.put(new Put(ROW).addColumn(FAMILY, QUALIFIER, VALUE)).get();
       assertTrue(errorTriggered.get());
       errorTriggered.set(false);

http://git-wip-us.apache.org/repos/asf/hbase/blob/db66e6cc/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableLocatePrefetch.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableLocatePrefetch.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableLocatePrefetch.java
index 13d8000..6c6bb98 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableLocatePrefetch.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableLocatePrefetch.java
@@ -69,8 +69,8 @@ public class TestAsyncTableLocatePrefetch {
 
   @Test
   public void test() throws InterruptedException, ExecutionException {
-    assertNotNull(LOCATOR
-      .getRegionLocation(TABLE_NAME, Bytes.toBytes("zzz"), RegionLocateType.CURRENT, false).get());
+    assertNotNull(LOCATOR.getRegionLocations(TABLE_NAME, Bytes.toBytes("zzz"),
+      RegionReplicaUtil.DEFAULT_REPLICA_ID, RegionLocateType.CURRENT, false).get());
     // we finish the request before we adding the remaining results to cache so sleep a bit here
     Thread.sleep(1000);
     // confirm that the locations of all the regions have been cached.

http://git-wip-us.apache.org/repos/asf/hbase/blob/db66e6cc/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRegionReplicasGet.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRegionReplicasGet.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRegionReplicasGet.java
new file mode 100644
index 0000000..0445a0e
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRegionReplicasGet.java
@@ -0,0 +1,204 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.ForkJoinPool;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Supplier;
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.RegionObserver;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+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({ MediumTests.class, ClientTests.class })
+public class TestAsyncTableRegionReplicasGet {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestAsyncTableRegionReplicasGet.class);
+
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+
+  private static TableName TABLE_NAME = TableName.valueOf("async");
+
+  private static byte[] FAMILY = Bytes.toBytes("cf");
+
+  private static byte[] QUALIFIER = Bytes.toBytes("cq");
+
+  private static byte[] ROW = Bytes.toBytes("row");
+
+  private static byte[] VALUE = Bytes.toBytes("value");
+
+  private static AsyncConnection ASYNC_CONN;
+
+  @Rule
+  public TestName testName = new TestName();
+
+  @Parameter
+  public Supplier<AsyncTable<?>> getTable;
+
+  private static AsyncTable<?> getRawTable() {
+    return ASYNC_CONN.getTable(TABLE_NAME);
+  }
+
+  private static AsyncTable<?> getTable() {
+    return ASYNC_CONN.getTable(TABLE_NAME, ForkJoinPool.commonPool());
+  }
+
+  @Parameters
+  public static List<Object[]> params() {
+    return Arrays.asList(new Supplier<?>[] { TestAsyncTableRegionReplicasGet::getRawTable },
+      new Supplier<?>[] { TestAsyncTableRegionReplicasGet::getTable });
+  }
+
+  private static volatile boolean FAIL_PRIMARY_GET = false;
+
+  private static AtomicInteger PRIMARY_GET_COUNT = new AtomicInteger(0);
+
+  private static AtomicInteger SECONDARY_GET_COUNT = new AtomicInteger(0);
+
+  public static final class FailPrimaryGetCP implements RegionObserver, RegionCoprocessor {
+
+    @Override
+    public Optional<RegionObserver> getRegionObserver() {
+      return Optional.of(this);
+    }
+
+    @Override
+    public void preGetOp(ObserverContext<RegionCoprocessorEnvironment> c, Get get,
+        List<Cell> result) throws IOException {
+      RegionInfo region = c.getEnvironment().getRegionInfo();
+      if (!region.getTable().equals(TABLE_NAME)) {
+        return;
+      }
+      if (region.getReplicaId() != RegionReplicaUtil.DEFAULT_REPLICA_ID) {
+        SECONDARY_GET_COUNT.incrementAndGet();
+      } else {
+        PRIMARY_GET_COUNT.incrementAndGet();
+        if (FAIL_PRIMARY_GET) {
+          throw new IOException("Inject error");
+        }
+      }
+    }
+  }
+
+  private static boolean allReplicasHaveRow() throws IOException {
+    for (RegionServerThread t : TEST_UTIL.getMiniHBaseCluster().getRegionServerThreads()) {
+      for (HRegion region : t.getRegionServer().getRegions(TABLE_NAME)) {
+        if (region.get(new Get(ROW), false).isEmpty()) {
+          return false;
+        }
+      }
+    }
+    return true;
+  }
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    // 10 mins
+    TEST_UTIL.getConfiguration().setLong(HConstants.HBASE_RPC_READ_TIMEOUT_KEY,
+      TimeUnit.MINUTES.toMillis(10));
+    // 1 second
+    TEST_UTIL.getConfiguration().setLong(ConnectionConfiguration.PRIMARY_CALL_TIMEOUT_MICROSECOND,
+      TimeUnit.SECONDS.toMicros(1));
+    // set a small pause so we will retry very quickly
+    TEST_UTIL.getConfiguration().setLong(HConstants.HBASE_CLIENT_PAUSE, 10);
+    // infinite retry
+    TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, Integer.MAX_VALUE);
+    TEST_UTIL.startMiniCluster(3);
+    TEST_UTIL.getAdmin()
+      .createTable(TableDescriptorBuilder.newBuilder(TABLE_NAME)
+        .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY)).setRegionReplication(3)
+        .setCoprocessor(FailPrimaryGetCP.class.getName()).build());
+    TEST_UTIL.waitUntilAllRegionsAssigned(TABLE_NAME);
+    ASYNC_CONN = ConnectionFactory.createAsyncConnection(TEST_UTIL.getConfiguration()).get();
+    AsyncTable<?> table = ASYNC_CONN.getTable(TABLE_NAME);
+    table.put(new Put(ROW).addColumn(FAMILY, QUALIFIER, VALUE)).get();
+    // this is the fastest way to let all replicas have the row
+    TEST_UTIL.getAdmin().disableTable(TABLE_NAME);
+    TEST_UTIL.getAdmin().enableTable(TABLE_NAME);
+    TEST_UTIL.waitFor(30000, () -> allReplicasHaveRow());
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    IOUtils.closeQuietly(ASYNC_CONN);
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  @Test
+  public void testNoReplicaRead() throws Exception {
+    FAIL_PRIMARY_GET = false;
+    SECONDARY_GET_COUNT.set(0);
+    AsyncTable<?> table = getTable.get();
+    Get get = new Get(ROW).setConsistency(Consistency.TIMELINE);
+    for (int i = 0; i < 1000; i++) {
+      assertArrayEquals(VALUE, table.get(get).get().getValue(FAMILY, QUALIFIER));
+    }
+    // the primary region is fine and the primary timeout is 1 second which is long enough, so we
+    // should not send any requests to secondary replicas even if the consistency is timeline.
+    Thread.sleep(5000);
+    assertEquals(0, SECONDARY_GET_COUNT.get());
+  }
+
+  @Test
+  public void testReplicaRead() throws Exception {
+    // fail the primary get request
+    FAIL_PRIMARY_GET = true;
+    Get get = new Get(ROW).setConsistency(Consistency.TIMELINE);
+    // make sure that we could still get the value from secondary replicas
+    AsyncTable<?> table = getTable.get();
+    assertArrayEquals(VALUE, table.get(get).get().getValue(FAMILY, QUALIFIER));
+    // make sure that the primary request has been canceled
+    Thread.sleep(5000);
+    int count = PRIMARY_GET_COUNT.get();
+    Thread.sleep(10000);
+    assertEquals(count, PRIMARY_GET_COUNT.get());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/db66e6cc/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKAsyncRegistry.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKAsyncRegistry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKAsyncRegistry.java
index db7546f..46890d0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKAsyncRegistry.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKAsyncRegistry.java
@@ -25,7 +25,6 @@ import static org.junit.Assert.assertNotSame;
 
 import java.io.IOException;
 import java.util.concurrent.ExecutionException;
-import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.stream.IntStream;
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
@@ -35,7 +34,6 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.RegionLocations;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.zookeeper.ReadOnlyZKClient;
@@ -52,43 +50,13 @@ public class TestZKAsyncRegistry {
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestZKAsyncRegistry.class);
+    HBaseClassTestRule.forClass(TestZKAsyncRegistry.class);
 
-  private static final Logger LOG = LoggerFactory.getLogger(TestZKAsyncRegistry.class);
-  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  static final Logger LOG = LoggerFactory.getLogger(TestZKAsyncRegistry.class);
+  static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 
   private static ZKAsyncRegistry REGISTRY;
 
-  // waits for all replicas to have region location
-  static void waitUntilAllReplicasHavingRegionLocation(TableName tbl) throws IOException {
-    TEST_UTIL.waitFor(
-      TEST_UTIL.getConfiguration().getLong("hbase.client.sync.wait.timeout.msec", 60000), 200, true,
-      new ExplainingPredicate<IOException>() {
-        @Override
-        public String explainFailure() throws IOException {
-          return TEST_UTIL.explainTableAvailability(tbl);
-        }
-
-        @Override
-        public boolean evaluate() throws IOException {
-          AtomicBoolean ready = new AtomicBoolean(true);
-          try {
-            RegionLocations locs = REGISTRY.getMetaRegionLocation().get();
-            assertEquals(3, locs.getRegionLocations().length);
-            IntStream.range(0, 3).forEach(i -> {
-              HRegionLocation loc = locs.getRegionLocation(i);
-              if (loc == null) {
-                ready.set(false);
-              }
-            });
-          } catch (Exception e) {
-            ready.set(false);
-          }
-          return ready.get();
-        }
-      });
-  }
-
   @BeforeClass
   public static void setUp() throws Exception {
     TEST_UTIL.getConfiguration().setInt(META_REPLICAS_NUM, 3);
@@ -107,14 +75,14 @@ public class TestZKAsyncRegistry {
     LOG.info("STARTED TEST");
     String clusterId = REGISTRY.getClusterId().get();
     String expectedClusterId = TEST_UTIL.getHBaseCluster().getMaster().getClusterId();
-    assertEquals("Expected " + expectedClusterId + ", found=" + clusterId,
-        expectedClusterId, clusterId);
+    assertEquals("Expected " + expectedClusterId + ", found=" + clusterId, expectedClusterId,
+      clusterId);
     assertEquals(TEST_UTIL.getHBaseCluster().getClusterMetrics().getLiveServerMetrics().size(),
       REGISTRY.getCurrentNrHRS().get().intValue());
     assertEquals(TEST_UTIL.getHBaseCluster().getMaster().getServerName(),
       REGISTRY.getMasterAddress().get());
     assertEquals(-1, REGISTRY.getMasterInfoPort().get().intValue());
-    waitUntilAllReplicasHavingRegionLocation(TableName.META_TABLE_NAME);
+    RegionReplicaTestHelper.waitUntilAllMetaReplicasHavingRegionLocation(REGISTRY, 3);
     RegionLocations locs = REGISTRY.getMetaRegionLocation().get();
     assertEquals(3, locs.getRegionLocations().length);
     IntStream.range(0, 3).forEach(i -> {


[12/15] hbase git commit: HBASE-21515 Also initialize an AsyncClusterConnection in HRegionServer

Posted by zh...@apache.org.
HBASE-21515 Also initialize an AsyncClusterConnection in HRegionServer


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

Branch: refs/heads/HBASE-21512
Commit: 6fff6c81fe1658827159e85845c9d3e595797375
Parents: 3fbdd5b
Author: zhangduo <zh...@apache.org>
Authored: Fri Nov 30 08:23:47 2018 +0800
Committer: Duo Zhang <zh...@apache.org>
Committed: Fri Jan 4 14:31:41 2019 +0800

----------------------------------------------------------------------
 .../hbase/client/AsyncClusterConnection.java    | 38 ++++++++++++
 .../hbase/client/AsyncConnectionImpl.java       | 39 ++++++------
 .../hbase/client/ClusterConnectionFactory.java  | 63 ++++++++++++++++++++
 .../hadoop/hbase/client/ConnectionFactory.java  |  5 +-
 .../hadoop/hbase/util/ReflectionUtils.java      | 22 ++++---
 .../java/org/apache/hadoop/hbase/Server.java    | 20 +++++++
 .../org/apache/hadoop/hbase/master/HMaster.java |  3 +
 .../hbase/regionserver/HRegionServer.java       | 56 ++++++++++++-----
 .../regionserver/ReplicationSyncUp.java         |  6 ++
 .../hadoop/hbase/MockRegionServerServices.java  |  5 ++
 .../client/TestAsyncNonMetaRegionLocator.java   |  2 +-
 ...syncNonMetaRegionLocatorConcurrenyLimit.java |  2 +-
 .../client/TestAsyncRegionLocatorTimeout.java   |  2 +-
 ...TestAsyncSingleRequestRpcRetryingCaller.java |  4 +-
 .../hbase/client/TestAsyncTableNoncedRetry.java |  2 +-
 .../hbase/master/MockNoopMasterServices.java    |  6 ++
 .../hadoop/hbase/master/MockRegionServer.java   |  5 ++
 .../hbase/master/TestActiveMasterManager.java   |  6 ++
 .../hbase/master/cleaner/TestHFileCleaner.java  |  6 ++
 .../master/cleaner/TestHFileLinkCleaner.java    |  6 ++
 .../hbase/master/cleaner/TestLogsCleaner.java   |  6 ++
 .../cleaner/TestReplicationHFileCleaner.java    |  6 ++
 .../regionserver/TestHeapMemoryManager.java     |  6 ++
 .../hbase/regionserver/TestSplitLogWorker.java  |  6 ++
 .../hbase/regionserver/TestWALLockup.java       |  6 ++
 .../TestReplicationTrackerZKImpl.java           |  6 ++
 .../TestReplicationSourceManager.java           |  6 ++
 .../security/token/TestTokenAuthentication.java |  6 ++
 .../apache/hadoop/hbase/util/MockServer.java    |  6 ++
 29 files changed, 302 insertions(+), 50 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnection.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnection.java
new file mode 100644
index 0000000..c7dea25
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnection.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.client;
+
+import org.apache.hadoop.hbase.ipc.RpcClient;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * The asynchronous connection for internal usage.
+ */
+@InterfaceAudience.Private
+public interface AsyncClusterConnection extends AsyncConnection {
+
+  /**
+   * Get the nonce generator for this connection.
+   */
+  NonceGenerator getNonceGenerator();
+
+  /**
+   * Get the rpc client we used to communicate with other servers.
+   */
+  RpcClient getRpcClient();
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
index 361d5b2..188e830 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
@@ -21,48 +21,48 @@ import static org.apache.hadoop.hbase.client.ConnectionUtils.NO_NONCE_GENERATOR;
 import static org.apache.hadoop.hbase.client.ConnectionUtils.getStubKey;
 import static org.apache.hadoop.hbase.client.NonceGenerator.CLIENT_NONCES_ENABLED_KEY;
 
-import org.apache.hadoop.hbase.AuthUtil;
-import org.apache.hadoop.hbase.ChoreService;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
-import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer;
-
 import java.io.IOException;
+import java.net.SocketAddress;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
-
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.AuthUtil;
+import org.apache.hadoop.hbase.ChoreService;
 import org.apache.hadoop.hbase.MasterNotRunningException;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hbase.ipc.HBaseRpcController;
 import org.apache.hadoop.hbase.ipc.RpcClient;
 import org.apache.hadoop.hbase.ipc.RpcClientFactory;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.CollectionUtils;
+import org.apache.hadoop.hbase.util.Threads;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer;
+
 import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterService;
-import org.apache.hadoop.hbase.util.CollectionUtils;
-import org.apache.hadoop.hbase.util.Threads;
 
 /**
  * The implementation of AsyncConnection.
  */
 @InterfaceAudience.Private
-class AsyncConnectionImpl implements AsyncConnection {
+class AsyncConnectionImpl implements AsyncClusterConnection {
 
   private static final Logger LOG = LoggerFactory.getLogger(AsyncConnectionImpl.class);
 
@@ -105,7 +105,7 @@ class AsyncConnectionImpl implements AsyncConnection {
   private ChoreService authService;
 
   public AsyncConnectionImpl(Configuration conf, AsyncRegistry registry, String clusterId,
-      User user) {
+      SocketAddress localAddress, User user) {
     this.conf = conf;
     this.user = user;
     if (user.isLoginFromKeytab()) {
@@ -113,7 +113,7 @@ class AsyncConnectionImpl implements AsyncConnection {
     }
     this.connConf = new AsyncConnectionConfiguration(conf);
     this.registry = registry;
-    this.rpcClient = RpcClientFactory.createClient(conf, clusterId);
+    this.rpcClient = RpcClientFactory.createClient(conf, clusterId, localAddress, null);
     this.rpcControllerFactory = RpcControllerFactory.instantiate(conf);
     this.hostnameCanChange = conf.getBoolean(RESOLVE_HOSTNAME_ON_FAIL_KEY, true);
     this.rpcTimeout =
@@ -157,11 +157,16 @@ class AsyncConnectionImpl implements AsyncConnection {
   }
 
   // ditto
-  @VisibleForTesting
+  @Override
   public NonceGenerator getNonceGenerator() {
     return nonceGenerator;
   }
 
+  @Override
+  public RpcClient getRpcClient() {
+    return rpcClient;
+  }
+
   private ClientService.Interface createRegionServerStub(ServerName serverName) throws IOException {
     return ClientService.newStub(rpcClient.createRpcChannel(serverName, user, rpcTimeout));
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnectionFactory.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnectionFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnectionFactory.java
new file mode 100644
index 0000000..68c0630
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnectionFactory.java
@@ -0,0 +1,63 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.net.SocketAddress;
+import java.util.concurrent.ExecutionException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hbase.thirdparty.com.google.common.base.Throwables;
+
+/**
+ * The factory for creating {@link AsyncClusterConnection}.
+ */
+@InterfaceAudience.Private
+public final class ClusterConnectionFactory {
+
+  private ClusterConnectionFactory() {
+  }
+
+  /**
+   * Create a new {@link AsyncClusterConnection} instance.
+   * <p/>
+   * Unlike what we have done in {@link ConnectionFactory}, here we just return an
+   * {@link AsyncClusterConnection} instead of a {@link java.util.concurrent.CompletableFuture},
+   * which means this method could block on fetching the cluster id. This is just used to simplify
+   * the implementation, as when starting new region servers, we do not need to be event-driven. Can
+   * change later if we want a {@link java.util.concurrent.CompletableFuture} here.
+   */
+  public static AsyncClusterConnection createAsyncClusterConnection(Configuration conf,
+      SocketAddress localAddress, User user) throws IOException {
+    AsyncRegistry registry = AsyncRegistryFactory.getRegistry(conf);
+    String clusterId;
+    try {
+      clusterId = registry.getClusterId().get();
+    } catch (InterruptedException e) {
+      throw (IOException) new InterruptedIOException().initCause(e);
+    } catch (ExecutionException e) {
+      Throwable cause = e.getCause();
+      Throwables.propagateIfPossible(cause, IOException.class);
+      throw new IOException(cause);
+    }
+    return new AsyncConnectionImpl(conf, registry, clusterId, localAddress, user);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java
index e24af74..2ba732a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java
@@ -295,9 +295,8 @@ public class ConnectionFactory {
         AsyncConnectionImpl.class, AsyncConnection.class);
       try {
         future.complete(
-          user.runAs((PrivilegedExceptionAction<? extends AsyncConnection>)() ->
-            ReflectionUtils.newInstance(clazz, conf, registry, clusterId, user))
-        );
+          user.runAs((PrivilegedExceptionAction<? extends AsyncConnection>) () -> ReflectionUtils
+            .newInstance(clazz, conf, registry, clusterId, null, user)));
       } catch (Exception e) {
         future.completeExceptionally(e);
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ReflectionUtils.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ReflectionUtils.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ReflectionUtils.java
index a136846..268249d 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ReflectionUtils.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ReflectionUtils.java
@@ -83,15 +83,19 @@ public class ReflectionUtils {
 
       boolean match = true;
       for (int i = 0; i < ctorParamTypes.length && match; ++i) {
-        Class<?> paramType = paramTypes[i].getClass();
-        match = (!ctorParamTypes[i].isPrimitive()) ? ctorParamTypes[i].isAssignableFrom(paramType) :
-                  ((int.class.equals(ctorParamTypes[i]) && Integer.class.equals(paramType)) ||
-                   (long.class.equals(ctorParamTypes[i]) && Long.class.equals(paramType)) ||
-                   (double.class.equals(ctorParamTypes[i]) && Double.class.equals(paramType)) ||
-                   (char.class.equals(ctorParamTypes[i]) && Character.class.equals(paramType)) ||
-                   (short.class.equals(ctorParamTypes[i]) && Short.class.equals(paramType)) ||
-                   (boolean.class.equals(ctorParamTypes[i]) && Boolean.class.equals(paramType)) ||
-                   (byte.class.equals(ctorParamTypes[i]) && Byte.class.equals(paramType)));
+        if (paramTypes[i] == null) {
+          match = !ctorParamTypes[i].isPrimitive();
+        } else {
+          Class<?> paramType = paramTypes[i].getClass();
+          match = (!ctorParamTypes[i].isPrimitive()) ? ctorParamTypes[i].isAssignableFrom(paramType)
+            : ((int.class.equals(ctorParamTypes[i]) && Integer.class.equals(paramType)) ||
+              (long.class.equals(ctorParamTypes[i]) && Long.class.equals(paramType)) ||
+              (double.class.equals(ctorParamTypes[i]) && Double.class.equals(paramType)) ||
+              (char.class.equals(ctorParamTypes[i]) && Character.class.equals(paramType)) ||
+              (short.class.equals(ctorParamTypes[i]) && Short.class.equals(paramType)) ||
+              (boolean.class.equals(ctorParamTypes[i]) && Boolean.class.equals(paramType)) ||
+              (byte.class.equals(ctorParamTypes[i]) && Byte.class.equals(paramType)));
+        }
       }
 
       if (match) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/hbase-server/src/main/java/org/apache/hadoop/hbase/Server.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/Server.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/Server.java
index fb898ea..c33d5af 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/Server.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/Server.java
@@ -20,6 +20,8 @@ package org.apache.hadoop.hbase;
 import java.io.IOException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
+import org.apache.hadoop.hbase.client.AsyncConnection;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
@@ -61,6 +63,24 @@ public interface Server extends Abortable, Stoppable {
   ClusterConnection getClusterConnection();
 
   /**
+   * Returns a reference to the servers' async connection.
+   * <p/>
+   * Important note: this method returns a reference to Connection which is managed by Server
+   * itself, so callers must NOT attempt to close connection obtained.
+   */
+  default AsyncConnection getAsyncConnection() {
+    return getAsyncClusterConnection();
+  }
+
+  /**
+   * Returns a reference to the servers' async cluster connection.
+   * <p/>
+   * Important note: this method returns a reference to Connection which is managed by Server
+   * itself, so callers must NOT attempt to close connection obtained.
+   */
+  AsyncClusterConnection getAsyncClusterConnection();
+
+  /**
    * @return The unique server name for this server.
    */
   ServerName getServerName();

http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index 0bcef59..52005d6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -3008,6 +3008,9 @@ public class HMaster extends HRegionServer implements MasterServices {
     if (this.clusterConnection != null) {
       this.clusterConnection.close();
     }
+    if (this.asyncClusterConnection != null) {
+      this.asyncClusterConnection.close();
+    }
   }
 
   public void stopMaster() throws IOException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index 13f277b..375b3f8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -77,7 +77,9 @@ import org.apache.hadoop.hbase.TableDescriptors;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.YouAreDeadException;
 import org.apache.hadoop.hbase.ZNodeClearer;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.client.ClusterConnectionFactory;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionUtils;
 import org.apache.hadoop.hbase.client.RegionInfo;
@@ -105,7 +107,6 @@ import org.apache.hadoop.hbase.io.util.MemorySizeUtil;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
 import org.apache.hadoop.hbase.ipc.NettyRpcClientConfigHelper;
 import org.apache.hadoop.hbase.ipc.RpcClient;
-import org.apache.hadoop.hbase.ipc.RpcClientFactory;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.ipc.RpcServer;
 import org.apache.hadoop.hbase.ipc.RpcServerInterface;
@@ -263,6 +264,11 @@ public class HRegionServer extends HasThread implements
   protected ClusterConnection clusterConnection;
 
   /**
+   * The asynchronous cluster connection to be shared by services.
+   */
+  protected AsyncClusterConnection asyncClusterConnection;
+
+  /**
    * Go here to get table descriptors.
    */
   protected TableDescriptors tableDescriptors;
@@ -776,11 +782,7 @@ public class HRegionServer extends HasThread implements
     return true;
   }
 
-  /**
-   * Create a 'smarter' Connection, one that is capable of by-passing RPC if the request is to the
-   * local server; i.e. a short-circuit Connection. Safe to use going to local or remote server.
-   */
-  private ClusterConnection createClusterConnection() throws IOException {
+  private Configuration unsetClientZookeeperQuorum() {
     Configuration conf = this.conf;
     if (conf.get(HConstants.CLIENT_ZOOKEEPER_QUORUM) != null) {
       // Use server ZK cluster for server-issued connections, so we clone
@@ -788,11 +790,20 @@ public class HRegionServer extends HasThread implements
       conf = new Configuration(this.conf);
       conf.unset(HConstants.CLIENT_ZOOKEEPER_QUORUM);
     }
+    return conf;
+  }
+
+  /**
+   * Create a 'smarter' Connection, one that is capable of by-passing RPC if the request is to the
+   * local server; i.e. a short-circuit Connection. Safe to use going to local or remote server.
+   */
+  private ClusterConnection createClusterConnection() throws IOException {
     // Create a cluster connection that when appropriate, can short-circuit and go directly to the
     // local server if the request is to the local server bypassing RPC. Can be used for both local
     // and remote invocations.
-    ClusterConnection conn = ConnectionUtils.createShortCircuitConnection(conf, null,
-      userProvider.getCurrent(), serverName, rpcServices, rpcServices);
+    ClusterConnection conn =
+      ConnectionUtils.createShortCircuitConnection(unsetClientZookeeperQuorum(), null,
+        userProvider.getCurrent(), serverName, rpcServices, rpcServices);
     // This is used to initialize the batch thread pool inside the connection implementation.
     // When deploy a fresh cluster, we may first use the cluster connection in InitMetaProcedure,
     // which will be executed inside the PEWorker, and then the batch thread pool will inherit the
@@ -826,9 +837,12 @@ public class HRegionServer extends HasThread implements
   /**
    * Setup our cluster connection if not already initialized.
    */
-  protected synchronized void setupClusterConnection() throws IOException {
+  protected final synchronized void setupClusterConnection() throws IOException {
     if (clusterConnection == null) {
       clusterConnection = createClusterConnection();
+      asyncClusterConnection =
+        ClusterConnectionFactory.createAsyncClusterConnection(unsetClientZookeeperQuorum(),
+          new InetSocketAddress(this.rpcServices.isa.getAddress(), 0), userProvider.getCurrent());
     }
   }
 
@@ -842,8 +856,7 @@ public class HRegionServer extends HasThread implements
       initializeZooKeeper();
       setupClusterConnection();
       // Setup RPC client for master communication
-      this.rpcClient = RpcClientFactory.createClient(conf, clusterId, new InetSocketAddress(
-          this.rpcServices.isa.getAddress(), 0), clusterConnection.getConnectionMetrics());
+      this.rpcClient = asyncClusterConnection.getRpcClient();
     } catch (Throwable t) {
       // Call stop if error or process will stick around for ever since server
       // puts up non-daemon threads.
@@ -1107,7 +1120,15 @@ public class HRegionServer extends HasThread implements
         LOG.warn("Attempt to close server's short circuit ClusterConnection failed.", e);
       }
     }
-
+    if (this.asyncClusterConnection != null) {
+      try {
+        this.asyncClusterConnection.close();
+      } catch (IOException e) {
+        // Although the {@link Closeable} interface throws an {@link
+        // IOException}, in reality, the implementation would never do that.
+        LOG.warn("Attempt to close server's AsyncClusterConnection failed.", e);
+      }
+    }
     // Closing the compactSplit thread before closing meta regions
     if (!this.killed && containsMetaTableRegions()) {
       if (!abortRequested || this.fsOk) {
@@ -3737,9 +3758,9 @@ public class HRegionServer extends HasThread implements
   }
 
   @Override
-  public EntityLock regionLock(List<RegionInfo> regionInfos, String description,
-      Abortable abort) throws IOException {
-    return new LockServiceClient(conf, lockStub, clusterConnection.getNonceGenerator())
+  public EntityLock regionLock(List<RegionInfo> regionInfos, String description, Abortable abort)
+      throws IOException {
+    return new LockServiceClient(conf, lockStub, asyncClusterConnection.getNonceGenerator())
       .regionLock(regionInfos, description, abort);
   }
 
@@ -3843,4 +3864,9 @@ public class HRegionServer extends HasThread implements
       System.exit(1);
     }
   }
+
+  @Override
+  public AsyncClusterConnection getAsyncClusterConnection() {
+    return asyncClusterConnection;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
index c7bccb3..7d1245c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.util.FSUtils;
@@ -180,5 +181,10 @@ public class ReplicationSyncUp extends Configured implements Tool {
     public Connection createConnection(Configuration conf) throws IOException {
       return null;
     }
+
+    @Override
+    public AsyncClusterConnection getAsyncClusterConnection() {
+      return null;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
index 0e4f241..5205960 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
@@ -31,6 +31,7 @@ import java.util.concurrent.ConcurrentSkipListMap;
 import java.util.concurrent.atomic.AtomicBoolean;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.RegionInfo;
@@ -368,4 +369,8 @@ public class MockRegionServerServices implements RegionServerServices {
   public Optional<MobFileCache> getMobFileCache() {
     return Optional.empty();
   }
+
+  public AsyncClusterConnection getAsyncClusterConnection() {
+    return null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java
index eeaf99f..550a6f3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java
@@ -81,7 +81,7 @@ public class TestAsyncNonMetaRegionLocator {
     TEST_UTIL.getAdmin().balancerSwitch(false, true);
     AsyncRegistry registry = AsyncRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
     CONN = new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), registry,
-      registry.getClusterId().get(), User.getCurrent());
+      registry.getClusterId().get(), null, User.getCurrent());
     LOCATOR = new AsyncNonMetaRegionLocator(CONN);
     SPLIT_KEYS = new byte[8][];
     for (int i = 111; i < 999; i += 111) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java
index 8cdb4a9..7e06218 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java
@@ -125,7 +125,7 @@ public class TestAsyncNonMetaRegionLocatorConcurrenyLimit {
     TEST_UTIL.getAdmin().balancerSwitch(false, true);
     AsyncRegistry registry = AsyncRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
     CONN = new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), registry,
-      registry.getClusterId().get(), User.getCurrent());
+      registry.getClusterId().get(), null, User.getCurrent());
     LOCATOR = new AsyncNonMetaRegionLocator(CONN);
     SPLIT_KEYS = IntStream.range(1, 256).mapToObj(i -> Bytes.toBytes(String.format("%02x", i)))
       .toArray(byte[][]::new);

http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTimeout.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTimeout.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTimeout.java
index 758aa30..0e28f96 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTimeout.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTimeout.java
@@ -96,7 +96,7 @@ public class TestAsyncRegionLocatorTimeout {
     TEST_UTIL.waitTableAvailable(TABLE_NAME);
     AsyncRegistry registry = AsyncRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
     CONN = new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), registry,
-        registry.getClusterId().get(), User.getCurrent());
+      registry.getClusterId().get(), null, User.getCurrent());
     LOCATOR = CONN.getLocator();
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java
index 7d8956b..29dcd56 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java
@@ -73,7 +73,7 @@ public class TestAsyncSingleRequestRpcRetryingCaller {
     TEST_UTIL.waitTableAvailable(TABLE_NAME);
     AsyncRegistry registry = AsyncRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
     CONN = new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), registry,
-      registry.getClusterId().get(), User.getCurrent());
+      registry.getClusterId().get(), null, User.getCurrent());
   }
 
   @AfterClass
@@ -164,7 +164,7 @@ public class TestAsyncSingleRequestRpcRetryingCaller {
         }
       };
     try (AsyncConnectionImpl mockedConn = new AsyncConnectionImpl(CONN.getConfiguration(),
-      CONN.registry, CONN.registry.getClusterId().get(), User.getCurrent()) {
+      CONN.registry, CONN.registry.getClusterId().get(), null, User.getCurrent()) {
 
       @Override
       AsyncRegionLocator getLocator() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableNoncedRetry.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableNoncedRetry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableNoncedRetry.java
index 3008561..e1e55f5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableNoncedRetry.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableNoncedRetry.java
@@ -85,7 +85,7 @@ public class TestAsyncTableNoncedRetry {
     TEST_UTIL.waitTableAvailable(TABLE_NAME);
     AsyncRegistry registry = AsyncRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
     ASYNC_CONN = new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), registry,
-        registry.getClusterId().get(), User.getCurrent()) {
+      registry.getClusterId().get(), null, User.getCurrent()) {
 
       @Override
       public NonceGenerator getNonceGenerator() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
index 9c55f57..3ebad66 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.CoordinatedStateManager;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableDescriptors;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.Connection;
@@ -473,4 +474,9 @@ public class MockNoopMasterServices implements MasterServices {
   public SyncReplicationReplayWALManager getSyncReplicationReplayWALManager() {
     return null;
   }
+
+  @Override
+  public AsyncClusterConnection getAsyncClusterConnection() {
+    return null;
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
index a930d7f..73d53c7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableDescriptors;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.RegionInfo;
@@ -721,4 +722,8 @@ class MockRegionServer implements AdminProtos.AdminService.BlockingInterface,
   public Optional<MobFileCache> getMobFileCache() {
     return Optional.empty();
   }
+
+  public AsyncClusterConnection getAsyncClusterConnection() {
+    return null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java
index 2300f54..77667a7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
@@ -349,5 +350,10 @@ public class TestActiveMasterManager {
     public Connection createConnection(Configuration conf) throws IOException {
       return null;
     }
+
+    @Override
+    public AsyncClusterConnection getAsyncClusterConnection() {
+      return null;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
index 5c8db3e..c5fad32 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
@@ -279,6 +280,11 @@ public class TestHFileCleaner {
     public Connection createConnection(Configuration conf) throws IOException {
       return null;
     }
+
+    @Override
+    public AsyncClusterConnection getAsyncClusterConnection() {
+      return null;
+    }
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java
index 119194b..fd11ff8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.io.HFileLink;
@@ -213,5 +214,10 @@ public class TestHFileLinkCleaner {
     public Connection createConnection(Configuration conf) throws IOException {
       return null;
     }
+
+    @Override
+    public AsyncClusterConnection getAsyncClusterConnection() {
+      return null;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
index 247ed01..3286032 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
@@ -45,6 +45,7 @@ import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.Waiter;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.master.HMaster;
@@ -409,6 +410,11 @@ public class TestLogsCleaner {
     public Connection createConnection(Configuration conf) throws IOException {
       return null;
     }
+
+    @Override
+    public AsyncClusterConnection getAsyncClusterConnection() {
+      return null;
+    }
   }
 
   static class FaultyZooKeeperWatcher extends ZKWatcher {

http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java
index d162bf3..9791643 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.master.HMaster;
@@ -303,6 +304,11 @@ public class TestReplicationHFileCleaner {
     public Connection createConnection(Configuration conf) throws IOException {
       return null;
     }
+
+    @Override
+    public AsyncClusterConnection getAsyncClusterConnection() {
+      return null;
+    }
   }
 
   static class FaultyZooKeeperWatcher extends ZKWatcher {

http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java
index 8c9ce75..4a359e4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.Waiter;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.io.hfile.BlockCache;
@@ -862,6 +863,11 @@ public class TestHeapMemoryManager {
     public Connection createConnection(Configuration conf) throws IOException {
       return null;
     }
+
+    @Override
+    public AsyncClusterConnection getAsyncClusterConnection() {
+      return null;
+    }
   }
 
   static class CustomHeapMemoryTuner implements HeapMemoryTuner {

http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java
index cbf932c..5481ff8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.SplitLogCounters;
 import org.apache.hadoop.hbase.SplitLogTask;
 import org.apache.hadoop.hbase.Waiter;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.coordination.ZkCoordinatedStateManager;
@@ -159,6 +160,11 @@ public class TestSplitLogWorker {
     public Connection createConnection(Configuration conf) throws IOException {
       return null;
     }
+
+    @Override
+    public AsyncClusterConnection getAsyncClusterConnection() {
+      return null;
+    }
   }
 
   private void waitForCounter(LongAdder ctr, long oldval, long newval, long timems)

http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
index 0e20252..9e9d1d6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Durability;
@@ -523,6 +524,11 @@ public class TestWALLockup {
     public Connection createConnection(Configuration conf) throws IOException {
       return null;
     }
+
+    @Override
+    public AsyncClusterConnection getAsyncClusterConnection() {
+      return null;
+    }
   }
 
   static class DummyWALActionsListener implements WALActionsListener {

http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java
index 863d558..62ab265 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -263,5 +264,10 @@ public class TestReplicationTrackerZKImpl {
     public Connection createConnection(Configuration conf) throws IOException {
       return null;
     }
+
+    @Override
+    public AsyncClusterConnection getAsyncClusterConnection() {
+      return null;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
index 86bbb09..427f319 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
@@ -57,6 +57,7 @@ import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Waiter;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Connection;
@@ -906,5 +907,10 @@ public abstract class TestReplicationSourceManager {
     public Connection createConnection(Configuration conf) throws IOException {
       return null;
     }
+
+    @Override
+    public AsyncClusterConnection getAsyncClusterConnection() {
+      return null;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java
index e4780f1..92c8e54 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java
@@ -44,6 +44,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
@@ -363,6 +364,11 @@ public class TestTokenAuthentication {
     public Connection createConnection(Configuration conf) throws IOException {
       return null;
     }
+
+    @Override
+    public AsyncClusterConnection getAsyncClusterConnection() {
+      return null;
+    }
   }
 
   @Parameters(name = "{index}: rpcServerImpl={0}")

http://git-wip-us.apache.org/repos/asf/hbase/blob/6fff6c81/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockServer.java
index c25db01..13212d2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockServer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockServer.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.log.HBaseMarkers;
@@ -143,4 +144,9 @@ public class MockServer implements Server {
   public Connection createConnection(Configuration conf) throws IOException {
     return null;
   }
+
+  @Override
+  public AsyncClusterConnection getAsyncClusterConnection() {
+    return null;
+  }
 }


[13/15] hbase git commit: HBASE-21516 Use AsyncConnection instead of Connection in SecureBulkLoadManager

Posted by zh...@apache.org.
HBASE-21516 Use AsyncConnection instead of Connection in SecureBulkLoadManager


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

Branch: refs/heads/HBASE-21512
Commit: 721af37c0ce2777620334cfd6fbb14a430533704
Parents: 6fff6c8
Author: zhangduo <zh...@apache.org>
Authored: Sat Dec 1 21:15:48 2018 +0800
Committer: Duo Zhang <zh...@apache.org>
Committed: Fri Jan 4 14:31:41 2019 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/protobuf/ProtobufUtil.java     |  5 +-
 .../hbase/shaded/protobuf/ProtobufUtil.java     |  7 ++-
 .../hbase/regionserver/HRegionServer.java       |  2 +-
 .../regionserver/SecureBulkLoadManager.java     | 24 +++++----
 .../hadoop/hbase/security/token/TokenUtil.java  | 57 +++++++++++++++-----
 .../hbase/security/token/TestTokenUtil.java     | 42 +++++++++++----
 6 files changed, 96 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/721af37c/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
index a3d49b5..d9e620b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
@@ -261,13 +261,12 @@ public final class ProtobufUtil {
    * just {@link ServiceException}. Prefer this method to
    * {@link #getRemoteException(ServiceException)} because trying to
    * contain direct protobuf references.
-   * @param e
    */
-  public static IOException handleRemoteException(Exception e) {
+  public static IOException handleRemoteException(Throwable e) {
     return makeIOExceptionOfException(e);
   }
 
-  private static IOException makeIOExceptionOfException(Exception e) {
+  private static IOException makeIOExceptionOfException(Throwable e) {
     Throwable t = e;
     if (e instanceof ServiceException ||
         e instanceof org.apache.hbase.thirdparty.com.google.protobuf.ServiceException) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/721af37c/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
index fea81f1..de2fb7d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
@@ -40,7 +40,6 @@ import java.util.concurrent.TimeUnit;
 import java.util.function.Function;
 import java.util.regex.Pattern;
 import java.util.stream.Collectors;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.ByteBufferExtendedCell;
@@ -123,6 +122,7 @@ import org.apache.hbase.thirdparty.com.google.protobuf.Service;
 import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
 import org.apache.hbase.thirdparty.com.google.protobuf.TextFormat;
 import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
+
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionRequest;
@@ -343,13 +343,12 @@ public final class ProtobufUtil {
    * just {@link ServiceException}. Prefer this method to
    * {@link #getRemoteException(ServiceException)} because trying to
    * contain direct protobuf references.
-   * @param e
    */
-  public static IOException handleRemoteException(Exception e) {
+  public static IOException handleRemoteException(Throwable e) {
     return makeIOExceptionOfException(e);
   }
 
-  private static IOException makeIOExceptionOfException(Exception e) {
+  private static IOException makeIOExceptionOfException(Throwable e) {
     Throwable t = e;
     if (e instanceof ServiceException) {
       t = e.getCause();

http://git-wip-us.apache.org/repos/asf/hbase/blob/721af37c/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index 375b3f8..6e5fc9f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -1930,7 +1930,7 @@ public class HRegionServer extends HasThread implements
     if (!isStopped() && !isAborted()) {
       initializeThreads();
     }
-    this.secureBulkLoadManager = new SecureBulkLoadManager(this.conf, clusterConnection);
+    this.secureBulkLoadManager = new SecureBulkLoadManager(this.conf, asyncClusterConnection);
     this.secureBulkLoadManager.start();
 
     // Health checker thread.

http://git-wip-us.apache.org/repos/asf/hbase/blob/721af37c/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SecureBulkLoadManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SecureBulkLoadManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SecureBulkLoadManager.java
index 566a6b6..add6519 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SecureBulkLoadManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SecureBulkLoadManager.java
@@ -1,4 +1,4 @@
-/*
+/**
  * 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
@@ -28,7 +28,6 @@ import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.function.BiFunction;
 import java.util.function.Consumer;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -38,11 +37,12 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.AsyncConnection;
 import org.apache.hadoop.hbase.ipc.RpcServer;
 import org.apache.hadoop.hbase.regionserver.HRegion.BulkLoadListener;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.UserProvider;
+import org.apache.hadoop.hbase.security.token.AuthenticationTokenIdentifier;
 import org.apache.hadoop.hbase.security.token.FsDelegationToken;
 import org.apache.hadoop.hbase.security.token.TokenUtil;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -56,7 +56,9 @@ import org.apache.hadoop.security.token.Token;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadRequest;
@@ -111,9 +113,9 @@ public class SecureBulkLoadManager {
 
   private UserProvider userProvider;
   private ConcurrentHashMap<UserGroupInformation, Integer> ugiReferenceCounter;
-  private Connection conn;
+  private AsyncConnection conn;
 
-  SecureBulkLoadManager(Configuration conf, Connection conn) {
+  SecureBulkLoadManager(Configuration conf, AsyncConnection conn) {
     this.conf = conf;
     this.conn = conn;
   }
@@ -212,23 +214,23 @@ public class SecureBulkLoadManager {
       familyPaths.add(new Pair<>(el.getFamily().toByteArray(), el.getPath()));
     }
 
-    Token userToken = null;
+    Token<AuthenticationTokenIdentifier> userToken = null;
     if (userProvider.isHadoopSecurityEnabled()) {
-      userToken = new Token(request.getFsToken().getIdentifier().toByteArray(), request.getFsToken()
-              .getPassword().toByteArray(), new Text(request.getFsToken().getKind()), new Text(
-              request.getFsToken().getService()));
+      userToken = new Token<>(request.getFsToken().getIdentifier().toByteArray(),
+        request.getFsToken().getPassword().toByteArray(), new Text(request.getFsToken().getKind()),
+        new Text(request.getFsToken().getService()));
     }
     final String bulkToken = request.getBulkToken();
     User user = getActiveUser();
     final UserGroupInformation ugi = user.getUGI();
     if (userProvider.isHadoopSecurityEnabled()) {
       try {
-        Token tok = TokenUtil.obtainToken(conn);
+        Token<AuthenticationTokenIdentifier> tok = TokenUtil.obtainToken(conn).get();
         if (tok != null) {
           boolean b = ugi.addToken(tok);
           LOG.debug("token added " + tok + " for user " + ugi + " return=" + b);
         }
-      } catch (IOException ioe) {
+      } catch (Exception ioe) {
         LOG.warn("unable to add token", ioe);
       }
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/721af37c/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenUtil.java
index c54d905..28efb84 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenUtil.java
@@ -1,4 +1,4 @@
-/*
+/**
  * 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
@@ -15,27 +15,29 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hadoop.hbase.security.token;
 
+import com.google.protobuf.ByteString;
+import com.google.protobuf.ServiceException;
 import java.io.IOException;
 import java.lang.reflect.UndeclaredThrowableException;
 import java.security.PrivilegedExceptionAction;
-
-import com.google.protobuf.ByteString;
-import com.google.protobuf.ServiceException;
-
-import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
+import java.util.concurrent.CompletableFuture;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.AsyncConnection;
+import org.apache.hadoop.hbase.client.AsyncTable;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
 import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos;
+import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos.AuthenticationService;
+import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos.GetAuthenticationTokenRequest;
+import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos.GetAuthenticationTokenResponse;
 import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapreduce.Job;
@@ -45,6 +47,8 @@ import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+
 /**
  * Utility methods for obtaining authentication tokens.
  */
@@ -64,12 +68,39 @@ public class TokenUtil {
 
   /**
    * Obtain and return an authentication token for the current user.
+   * @param conn The async HBase cluster connection
+   * @return the authentication token instance, wrapped by a {@link CompletableFuture}.
+   */
+  public static CompletableFuture<Token<AuthenticationTokenIdentifier>> obtainToken(
+      AsyncConnection conn) {
+    CompletableFuture<Token<AuthenticationTokenIdentifier>> future = new CompletableFuture<>();
+    if (injectedException != null) {
+      future.completeExceptionally(injectedException);
+      return future;
+    }
+    AsyncTable<?> table = conn.getTable(TableName.META_TABLE_NAME);
+    table.<AuthenticationService.Interface, GetAuthenticationTokenResponse> coprocessorService(
+      AuthenticationProtos.AuthenticationService::newStub,
+      (s, c, r) -> s.getAuthenticationToken(c,
+        AuthenticationProtos.GetAuthenticationTokenRequest.getDefaultInstance(), r),
+      HConstants.EMPTY_START_ROW).whenComplete((resp, error) -> {
+        if (error != null) {
+          future.completeExceptionally(ProtobufUtil.handleRemoteException(error));
+        } else {
+          future.complete(toToken(resp.getToken()));
+        }
+      });
+    return future;
+  }
+
+  /**
+   * Obtain and return an authentication token for the current user.
    * @param conn The HBase cluster connection
    * @throws IOException if a remote error or serialization problem occurs.
    * @return the authentication token instance
    */
-  public static Token<AuthenticationTokenIdentifier> obtainToken(
-      Connection conn) throws IOException {
+  public static Token<AuthenticationTokenIdentifier> obtainToken(Connection conn)
+      throws IOException {
     Table meta = null;
     try {
       injectFault();
@@ -77,9 +108,9 @@ public class TokenUtil {
       meta = conn.getTable(TableName.META_TABLE_NAME);
       CoprocessorRpcChannel rpcChannel = meta.coprocessorService(HConstants.EMPTY_START_ROW);
       AuthenticationProtos.AuthenticationService.BlockingInterface service =
-          AuthenticationProtos.AuthenticationService.newBlockingStub(rpcChannel);
-      AuthenticationProtos.GetAuthenticationTokenResponse response = service.getAuthenticationToken(null,
-          AuthenticationProtos.GetAuthenticationTokenRequest.getDefaultInstance());
+        AuthenticationService.newBlockingStub(rpcChannel);
+      GetAuthenticationTokenResponse response =
+        service.getAuthenticationToken(null, GetAuthenticationTokenRequest.getDefaultInstance());
 
       return toToken(response.getToken());
     } catch (ServiceException se) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/721af37c/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenUtil.java
index 32fcddb..585a3ec 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenUtil.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenUtil.java
@@ -18,35 +18,53 @@
 package org.apache.hadoop.hbase.security.token;
 
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertSame;
 import static org.junit.Assert.fail;
 
+import java.io.IOException;
 import java.lang.reflect.Field;
 import java.lang.reflect.InvocationTargetException;
 import java.net.URL;
 import java.net.URLClassLoader;
-
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.client.AsyncConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.After;
+import org.junit.Before;
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import org.apache.hbase.thirdparty.com.google.common.io.Closeables;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 
 @Category(SmallTests.class)
 public class TestTokenUtil {
+
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestTokenUtil.class);
+    HBaseClassTestRule.forClass(TestTokenUtil.class);
 
-  @Test
-  public void testObtainToken() throws Exception {
+  private URLClassLoader cl;
+
+  @Before
+  public void setUp() {
     URL urlPU = ProtobufUtil.class.getProtectionDomain().getCodeSource().getLocation();
     URL urlTU = TokenUtil.class.getProtectionDomain().getCodeSource().getLocation();
+    cl = new URLClassLoader(new URL[] { urlPU, urlTU }, getClass().getClassLoader());
+  }
 
-    ClassLoader cl = new URLClassLoader(new URL[] { urlPU, urlTU }, getClass().getClassLoader());
+  @After
+  public void tearDown() throws IOException {
+    Closeables.close(cl, true);
+  }
 
+  @Test
+  public void testObtainToken() throws Exception {
     Throwable injected = new com.google.protobuf.ServiceException("injected");
 
     Class<?> tokenUtil = cl.loadClass(TokenUtil.class.getCanonicalName());
@@ -55,8 +73,7 @@ public class TestTokenUtil {
     shouldInjectFault.set(null, injected);
 
     try {
-      tokenUtil.getMethod("obtainToken", Connection.class)
-          .invoke(null, new Object[] { null });
+      tokenUtil.getMethod("obtainToken", Connection.class).invoke(null, new Object[] { null });
       fail("Should have injected exception.");
     } catch (InvocationTargetException e) {
       Throwable t = e;
@@ -72,9 +89,16 @@ public class TestTokenUtil {
       }
     }
 
+    CompletableFuture<?> future = (CompletableFuture<?>) tokenUtil
+      .getMethod("obtainToken", AsyncConnection.class).invoke(null, new Object[] { null });
+    try {
+      future.get();
+      fail("Should have injected exception.");
+    } catch (ExecutionException e) {
+      assertSame(injected, e.getCause());
+    }
     Boolean loaded = (Boolean) cl.loadClass(ProtobufUtil.class.getCanonicalName())
-        .getDeclaredMethod("isClassLoaderLoaded")
-        .invoke(null);
+      .getDeclaredMethod("isClassLoaderLoaded").invoke(null);
     assertFalse("Should not have loaded DynamicClassLoader", loaded);
   }
 }


[02/15] hbase git commit: HBASE-21652 Refactor ThriftServer making thrift2 server inherited from thrift1 server

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b6b4af/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftServer.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftServer.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftServer.java
index 5681569..fa3d39d 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftServer.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftServer.java
@@ -18,355 +18,86 @@
  */
 package org.apache.hadoop.hbase.thrift2;
 
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.net.UnknownHostException;
-import java.security.PrivilegedAction;
-import java.util.Map;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.SynchronousQueue;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
+import static org.apache.hadoop.hbase.thrift.Constants.READONLY_OPTION;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_READONLY_ENABLED;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_READONLY_ENABLED_DEFAULT;
 
-import javax.security.auth.callback.Callback;
-import javax.security.auth.callback.UnsupportedCallbackException;
-import javax.security.sasl.AuthorizeCallback;
-import javax.security.sasl.SaslServer;
+import java.io.IOException;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.hadoop.hbase.filter.ParseFilter;
-import org.apache.hadoop.hbase.http.InfoServer;
-import org.apache.hadoop.hbase.security.SaslUtil;
-import org.apache.hadoop.hbase.security.SecurityUtil;
 import org.apache.hadoop.hbase.security.UserProvider;
-import org.apache.hadoop.hbase.thrift.CallQueue;
-import org.apache.hadoop.hbase.thrift.THBaseThreadPoolExecutor;
-import org.apache.hadoop.hbase.thrift.ThriftMetrics;
+import org.apache.hadoop.hbase.thrift.HBaseServiceHandler;
+import org.apache.hadoop.hbase.thrift.HbaseHandlerMetricsProxy;
 import org.apache.hadoop.hbase.thrift2.generated.THBaseService;
-import org.apache.hadoop.hbase.util.DNS;
-import org.apache.hadoop.hbase.util.JvmPauseMonitor;
-import org.apache.hadoop.hbase.util.Strings;
-import org.apache.hadoop.security.SaslRpcServer.SaslGssCallbackHandler;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.util.ToolRunner;
-import org.apache.thrift.TException;
 import org.apache.thrift.TProcessor;
-import org.apache.thrift.protocol.TBinaryProtocol;
-import org.apache.thrift.protocol.TCompactProtocol;
-import org.apache.thrift.protocol.TProtocol;
-import org.apache.thrift.protocol.TProtocolFactory;
-import org.apache.thrift.server.THsHaServer;
-import org.apache.thrift.server.TNonblockingServer;
-import org.apache.thrift.server.TServer;
-import org.apache.thrift.server.TThreadPoolServer;
-import org.apache.thrift.server.TThreadedSelectorServer;
-import org.apache.thrift.transport.TFramedTransport;
-import org.apache.thrift.transport.TNonblockingServerSocket;
-import org.apache.thrift.transport.TNonblockingServerTransport;
-import org.apache.thrift.transport.TSaslServerTransport;
-import org.apache.thrift.transport.TServerSocket;
-import org.apache.thrift.transport.TServerTransport;
-import org.apache.thrift.transport.TTransportException;
-import org.apache.thrift.transport.TTransportFactory;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
+
 import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
-import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLineParser;
-import org.apache.hbase.thirdparty.org.apache.commons.cli.DefaultParser;
 import org.apache.hbase.thirdparty.org.apache.commons.cli.HelpFormatter;
-import org.apache.hbase.thirdparty.org.apache.commons.cli.Option;
-import org.apache.hbase.thirdparty.org.apache.commons.cli.OptionGroup;
 import org.apache.hbase.thirdparty.org.apache.commons.cli.Options;
-import org.apache.hbase.thirdparty.org.apache.commons.cli.ParseException;
 
 /**
  * ThriftServer - this class starts up a Thrift server which implements the HBase API specified in
  * the HbaseClient.thrift IDL file.
  */
+@edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "NM_SAME_SIMPLE_NAME_AS_SUPERCLASS",
+    justification = "Change the name will be an incompatible change, will do it later")
 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
 @SuppressWarnings({ "rawtypes", "unchecked" })
-public class ThriftServer extends Configured implements Tool {
+public class ThriftServer extends org.apache.hadoop.hbase.thrift.ThriftServer {
   private static final Logger log = LoggerFactory.getLogger(ThriftServer.class);
 
-  /**
-   * Thrift quality of protection configuration key. Valid values can be:
-   * privacy: authentication, integrity and confidentiality checking
-   * integrity: authentication and integrity checking
-   * authentication: authentication only
-   *
-   * This is used to authenticate the callers and support impersonation.
-   * The thrift server and the HBase cluster must run in secure mode.
-   */
-  static final String THRIFT_QOP_KEY = "hbase.thrift.security.qop";
-
-  static final String BACKLOG_CONF_KEY = "hbase.regionserver.thrift.backlog";
 
-  public static final int DEFAULT_LISTEN_PORT = 9090;
-
-  private static final String READ_TIMEOUT_OPTION = "readTimeout";
-
-  /**
-   * Amount of time in milliseconds before a server thread will timeout
-   * waiting for client to send data on a connected socket. Currently,
-   * applies only to TBoundedThreadPoolServer
-   */
-  public static final String THRIFT_SERVER_SOCKET_READ_TIMEOUT_KEY =
-    "hbase.thrift.server.socket.read.timeout";
-  public static final int THRIFT_SERVER_SOCKET_READ_TIMEOUT_DEFAULT = 60000;
-
-  public ThriftServer() {
+  public ThriftServer(Configuration conf) {
+    super(conf);
   }
 
-  private static void printUsage() {
+  @Override
+  protected void printUsageAndExit(Options options, int exitCode)
+      throws Shell.ExitCodeException {
     HelpFormatter formatter = new HelpFormatter();
-    formatter.printHelp("Thrift", null, getOptions(),
+    formatter.printHelp("Thrift", null, options,
         "To start the Thrift server run 'hbase-daemon.sh start thrift2' or " +
-        "'hbase thrift2'\n" +
-        "To shutdown the thrift server run 'hbase-daemon.sh stop thrift2' or" +
-        " send a kill signal to the thrift server pid",
+            "'hbase thrift2'\n" +
+            "To shutdown the thrift server run 'hbase-daemon.sh stop thrift2' or" +
+            " send a kill signal to the thrift server pid",
         true);
+    throw new Shell.ExitCodeException(exitCode, "");
   }
 
-  private static Options getOptions() {
-    Options options = new Options();
-    options.addOption("b", "bind", true,
-        "Address to bind the Thrift server to. [default: 0.0.0.0]");
-    options.addOption("p", "port", true, "Port to bind to [default: " + DEFAULT_LISTEN_PORT + "]");
-    options.addOption("f", "framed", false, "Use framed transport");
-    options.addOption("c", "compact", false, "Use the compact protocol");
-    options.addOption("w", "workers", true, "How many worker threads to use.");
-    options.addOption("s", "selectors", true, "How many selector threads to use.");
-    options.addOption("q", "callQueueSize", true,
-      "Max size of request queue (unbounded by default)");
-    options.addOption("h", "help", false, "Print help information");
-    options.addOption(null, "infoport", true, "Port for web UI");
-    options.addOption("t", READ_TIMEOUT_OPTION, true,
-      "Amount of time in milliseconds before a server thread will timeout " +
-      "waiting for client to send data on a connected socket. Currently, " +
-      "only applies to TBoundedThreadPoolServer");
-    options.addOption("ro", "readonly", false,
-      "Respond only to read method requests [default: false]");
-    OptionGroup servers = new OptionGroup();
-    servers.addOption(new Option("nonblocking", false,
-            "Use the TNonblockingServer. This implies the framed transport."));
-    servers.addOption(new Option("hsha", false,
-            "Use the THsHaServer. This implies the framed transport."));
-    servers.addOption(new Option("selector", false,
-            "Use the TThreadedSelectorServer. This implies the framed transport."));
-    servers.addOption(new Option("threadpool", false,
-            "Use the TThreadPoolServer. This is the default."));
-    options.addOptionGroup(servers);
-    return options;
-  }
-
-  private static CommandLine parseArguments(Configuration conf, Options options, String[] args)
-      throws ParseException, IOException {
-    CommandLineParser parser = new DefaultParser();
-    return parser.parse(options, args);
-  }
-
-  private static TProtocolFactory getTProtocolFactory(boolean isCompact) {
-    if (isCompact) {
-      log.debug("Using compact protocol");
-      return new TCompactProtocol.Factory();
-    } else {
-      log.debug("Using binary protocol");
-      return new TBinaryProtocol.Factory();
-    }
-  }
-
-  private static TTransportFactory getTTransportFactory(
-      SaslUtil.QualityOfProtection qop, String name, String host,
-      boolean framed, int frameSize) {
-    if (framed) {
-      if (qop != null) {
-        throw new RuntimeException("Thrift server authentication"
-          + " doesn't work with framed transport yet");
-      }
-      log.debug("Using framed transport");
-      return new TFramedTransport.Factory(frameSize);
-    } else if (qop == null) {
-      return new TTransportFactory();
-    } else {
-      Map<String, String> saslProperties = SaslUtil.initSaslProperties(qop.name());
-      TSaslServerTransport.Factory saslFactory = new TSaslServerTransport.Factory();
-      saslFactory.addServerDefinition("GSSAPI", name, host, saslProperties,
-        new SaslGssCallbackHandler() {
-          @Override
-          public void handle(Callback[] callbacks)
-              throws UnsupportedCallbackException {
-            AuthorizeCallback ac = null;
-            for (Callback callback : callbacks) {
-              if (callback instanceof AuthorizeCallback) {
-                ac = (AuthorizeCallback) callback;
-              } else {
-                throw new UnsupportedCallbackException(callback,
-                    "Unrecognized SASL GSSAPI Callback");
-              }
-            }
-            if (ac != null) {
-              String authid = ac.getAuthenticationID();
-              String authzid = ac.getAuthorizationID();
-              if (!authid.equals(authzid)) {
-                ac.setAuthorized(false);
-              } else {
-                ac.setAuthorized(true);
-                String userName = SecurityUtil.getUserFromPrincipal(authzid);
-                log.info("Effective user: " + userName);
-                ac.setAuthorizedID(userName);
-              }
-            }
-          }
-        });
-      return saslFactory;
-    }
-  }
-
-  /*
-   * If bindValue is null, we don't bind.
-   */
-  private static InetSocketAddress bindToPort(String bindValue, int listenPort)
-      throws UnknownHostException {
-    try {
-      if (bindValue == null) {
-        return new InetSocketAddress(listenPort);
-      } else {
-        return new InetSocketAddress(InetAddress.getByName(bindValue), listenPort);
-      }
-    } catch (UnknownHostException e) {
-      throw new RuntimeException("Could not bind to provided ip address", e);
-    }
-  }
-
-  private static TServer getTNonBlockingServer(TProtocolFactory protocolFactory,
-      TProcessor processor, TTransportFactory transportFactory, InetSocketAddress inetSocketAddress)
-          throws TTransportException {
-    TNonblockingServerTransport serverTransport = new TNonblockingServerSocket(inetSocketAddress);
-    log.info("starting HBase Nonblocking Thrift server on " + inetSocketAddress.toString());
-    TNonblockingServer.Args serverArgs = new TNonblockingServer.Args(serverTransport);
-    serverArgs.processor(processor);
-    serverArgs.transportFactory(transportFactory);
-    serverArgs.protocolFactory(protocolFactory);
-    return new TNonblockingServer(serverArgs);
-  }
-
-  private static TServer getTHsHaServer(TProtocolFactory protocolFactory,
-      TProcessor processor, TTransportFactory transportFactory,
-      int workerThreads, int maxCallQueueSize,
-      InetSocketAddress inetSocketAddress, ThriftMetrics metrics)
-      throws TTransportException {
-    TNonblockingServerTransport serverTransport = new TNonblockingServerSocket(inetSocketAddress);
-    log.info("starting HBase HsHA Thrift server on " + inetSocketAddress.toString());
-    THsHaServer.Args serverArgs = new THsHaServer.Args(serverTransport);
-    if (workerThreads > 0) {
-      // Could support the min & max threads, avoiding to preserve existing functionality.
-      serverArgs.minWorkerThreads(workerThreads).maxWorkerThreads(workerThreads);
-    }
-    ExecutorService executorService = createExecutor(
-        workerThreads, maxCallQueueSize, metrics);
-    serverArgs.executorService(executorService);
-    serverArgs.processor(processor);
-    serverArgs.transportFactory(transportFactory);
-    serverArgs.protocolFactory(protocolFactory);
-    return new THsHaServer(serverArgs);
-  }
-
-  private static TServer getTThreadedSelectorServer(TProtocolFactory protocolFactory,
-      TProcessor processor, TTransportFactory transportFactory,
-      int workerThreads, int selectorThreads, int maxCallQueueSize,
-      InetSocketAddress inetSocketAddress, ThriftMetrics metrics)
-      throws TTransportException {
-    TNonblockingServerTransport serverTransport = new TNonblockingServerSocket(inetSocketAddress);
-    log.info("starting HBase ThreadedSelector Thrift server on " + inetSocketAddress.toString());
-    TThreadedSelectorServer.Args serverArgs = new TThreadedSelectorServer.Args(serverTransport);
-    if (workerThreads > 0) {
-      serverArgs.workerThreads(workerThreads);
-    }
-    if (selectorThreads > 0) {
-      serverArgs.selectorThreads(selectorThreads);
-    }
-
-    ExecutorService executorService = createExecutor(
-        workerThreads, maxCallQueueSize, metrics);
-    serverArgs.executorService(executorService);
-    serverArgs.processor(processor);
-    serverArgs.transportFactory(transportFactory);
-    serverArgs.protocolFactory(protocolFactory);
-    return new TThreadedSelectorServer(serverArgs);
+  @Override
+  protected HBaseServiceHandler createHandler(Configuration conf, UserProvider userProvider)
+      throws IOException {
+    return new ThriftHBaseServiceHandler(conf, userProvider);
   }
 
-  private static ExecutorService createExecutor(
-      int workerThreads, int maxCallQueueSize, ThriftMetrics metrics) {
-    CallQueue callQueue;
-    if (maxCallQueueSize > 0) {
-      callQueue = new CallQueue(new LinkedBlockingQueue<>(maxCallQueueSize), metrics);
-    } else {
-      callQueue = new CallQueue(new LinkedBlockingQueue<>(), metrics);
-    }
-
-    ThreadFactoryBuilder tfb = new ThreadFactoryBuilder();
-    tfb.setDaemon(true);
-    tfb.setNameFormat("thrift2-worker-%d");
-    ThreadPoolExecutor pool = new THBaseThreadPoolExecutor(workerThreads, workerThreads,
-            Long.MAX_VALUE, TimeUnit.SECONDS, callQueue, tfb.build(), metrics);
-    pool.prestartAllCoreThreads();
-    return pool;
+  @Override
+  protected TProcessor createProcessor() {
+    return new THBaseService.Processor<>(HbaseHandlerMetricsProxy
+        .newInstance((THBaseService.Iface) hbaseServiceHandler, metrics, conf));
   }
 
-  private static TServer getTThreadPoolServer(TProtocolFactory protocolFactory,
-                                              TProcessor processor,
-                                              TTransportFactory transportFactory,
-                                              int workerThreads,
-                                              InetSocketAddress inetSocketAddress,
-                                              int backlog,
-                                              int clientTimeout,
-                                              ThriftMetrics metrics)
-      throws TTransportException {
-    TServerTransport serverTransport = new TServerSocket(
-                                           new TServerSocket.ServerSocketTransportArgs().
-                                               bindAddr(inetSocketAddress).backlog(backlog).
-                                               clientTimeout(clientTimeout));
-    log.info("starting HBase ThreadPool Thrift server on " + inetSocketAddress.toString());
-    TThreadPoolServer.Args serverArgs = new TThreadPoolServer.Args(serverTransport);
-    serverArgs.processor(processor);
-    serverArgs.transportFactory(transportFactory);
-    serverArgs.protocolFactory(protocolFactory);
-    if (workerThreads > 0) {
-      serverArgs.maxWorkerThreads(workerThreads);
-    }
-    ThreadPoolExecutor executor = new THBaseThreadPoolExecutor(serverArgs.minWorkerThreads,
-        serverArgs.maxWorkerThreads, serverArgs.stopTimeoutVal, TimeUnit.SECONDS,
-        new SynchronousQueue<>(), metrics);
-    serverArgs.executorService(executor);
-
-    return new TThreadPoolServer(serverArgs);
+  @Override
+  protected void addOptions(Options options) {
+    super.addOptions(options);
+    options.addOption("ro", READONLY_OPTION, false,
+        "Respond only to read method requests [default: false]");
   }
 
-  /**
-   * Adds the option to pre-load filters at startup.
-   *
-   * @param conf  The current configuration instance.
-   */
-  protected static void registerFilters(Configuration conf) {
-    String[] filters = conf.getStrings("hbase.thrift.filters");
-    if(filters != null) {
-      for(String filterClass: filters) {
-        String[] filterPart = filterClass.split(":");
-        if(filterPart.length != 2) {
-          log.warn("Invalid filter specification " + filterClass + " - skipping");
-        } else {
-          ParseFilter.registerFilter(filterPart[0], filterPart[1]);
-        }
-      }
+  @Override
+  protected void parseCommandLine(CommandLine cmd, Options options) throws Shell.ExitCodeException {
+    super.parseCommandLine(cmd, options);
+    boolean readOnly = THRIFT_READONLY_ENABLED_DEFAULT;
+    if (cmd.hasOption(READONLY_OPTION)) {
+      readOnly = true;
     }
+    conf.setBoolean(THRIFT_READONLY_ENABLED, readOnly);
   }
 
   /**
@@ -375,249 +106,8 @@ public class ThriftServer extends Configured implements Tool {
   public static void main(String[] args) throws Exception {
     final Configuration conf = HBaseConfiguration.create();
     // for now, only time we return is on an argument error.
-    final int status = ToolRunner.run(conf, new ThriftServer(), args);
+    final int status = ToolRunner.run(conf, new ThriftServer(conf), args);
     System.exit(status);
   }
 
-  @Override
-  public int run(String[] args) throws Exception {
-    final Configuration conf = getConf();
-    Options options = getOptions();
-    CommandLine cmd = parseArguments(conf, options, args);
-    int workerThreads = 0;
-    int selectorThreads = 0;
-    int maxCallQueueSize = -1; // use unbounded queue by default
-
-    if (cmd.hasOption("help")) {
-      printUsage();
-      return 1;
-    }
-
-    // Get address to bind
-    String bindAddress = getBindAddress(conf, cmd);
-
-    // check if server should only process read requests, if so override the conf
-    if (cmd.hasOption("readonly")) {
-      conf.setBoolean("hbase.thrift.readonly", true);
-      if (log.isDebugEnabled()) {
-        log.debug("readonly set to true");
-      }
-    }
-
-    // Get read timeout
-    int readTimeout = getReadTimeout(conf, cmd);
-    // Get port to bind to
-    int listenPort = getListenPort(conf, cmd);
-    // Thrift's implementation uses '0' as a placeholder for 'use the default.'
-    int backlog = conf.getInt(BACKLOG_CONF_KEY, 0);
-
-    // Local hostname and user name, used only if QOP is configured.
-    String host = null;
-    String name = null;
-
-    UserProvider userProvider = UserProvider.instantiate(conf);
-    // login the server principal (if using secure Hadoop)
-    boolean securityEnabled = userProvider.isHadoopSecurityEnabled()
-      && userProvider.isHBaseSecurityEnabled();
-    if (securityEnabled) {
-      host = Strings.domainNamePointerToHostName(DNS.getDefaultHost(
-        conf.get("hbase.thrift.dns.interface", "default"),
-        conf.get("hbase.thrift.dns.nameserver", "default")));
-      userProvider.login("hbase.thrift.keytab.file", "hbase.thrift.kerberos.principal", host);
-    }
-
-    UserGroupInformation realUser = userProvider.getCurrent().getUGI();
-    String stringQop = conf.get(THRIFT_QOP_KEY);
-    SaslUtil.QualityOfProtection qop = null;
-    if (stringQop != null) {
-      qop = SaslUtil.getQop(stringQop);
-      if (!securityEnabled) {
-        throw new IOException("Thrift server must run in secure mode to support authentication");
-      }
-      // Extract the name from the principal
-      name = SecurityUtil.getUserFromPrincipal(conf.get("hbase.thrift.kerberos.principal"));
-    }
-
-    boolean nonblocking = cmd.hasOption("nonblocking");
-    boolean hsha = cmd.hasOption("hsha");
-    boolean selector = cmd.hasOption("selector");
-
-    ThriftMetrics metrics = new ThriftMetrics(conf, ThriftMetrics.ThriftServerType.TWO);
-    final JvmPauseMonitor pauseMonitor = new JvmPauseMonitor(conf, metrics.getSource());
-
-    String implType = getImplType(nonblocking, hsha, selector);
-
-    conf.set("hbase.regionserver.thrift.server.type", implType);
-    conf.setInt("hbase.regionserver.thrift.port", listenPort);
-    registerFilters(conf);
-
-    // Construct correct ProtocolFactory
-    boolean compact = cmd.hasOption("compact") ||
-        conf.getBoolean("hbase.regionserver.thrift.compact", false);
-    TProtocolFactory protocolFactory = getTProtocolFactory(compact);
-    final ThriftHBaseServiceHandler hbaseHandler =
-      new ThriftHBaseServiceHandler(conf, userProvider);
-    THBaseService.Iface handler =
-      ThriftHBaseServiceHandler.newInstance(hbaseHandler, metrics);
-    final THBaseService.Processor p = new THBaseService.Processor(handler);
-    conf.setBoolean("hbase.regionserver.thrift.compact", compact);
-    TProcessor processor = p;
-
-    boolean framed = cmd.hasOption("framed") ||
-        conf.getBoolean("hbase.regionserver.thrift.framed", false) || nonblocking || hsha;
-    TTransportFactory transportFactory = getTTransportFactory(qop, name, host, framed,
-        conf.getInt("hbase.regionserver.thrift.framed.max_frame_size_in_mb", 2) * 1024 * 1024);
-    InetSocketAddress inetSocketAddress = bindToPort(bindAddress, listenPort);
-    conf.setBoolean("hbase.regionserver.thrift.framed", framed);
-    if (qop != null) {
-      // Create a processor wrapper, to get the caller
-      processor = new TProcessor() {
-        @Override
-        public boolean process(TProtocol inProt,
-            TProtocol outProt) throws TException {
-          TSaslServerTransport saslServerTransport =
-            (TSaslServerTransport)inProt.getTransport();
-          SaslServer saslServer = saslServerTransport.getSaslServer();
-          String principal = saslServer.getAuthorizationID();
-          hbaseHandler.setEffectiveUser(principal);
-          return p.process(inProt, outProt);
-        }
-      };
-    }
-
-    if (cmd.hasOption("w")) {
-      workerThreads = Integer.parseInt(cmd.getOptionValue("w"));
-    }
-    if (cmd.hasOption("s")) {
-      selectorThreads = Integer.parseInt(cmd.getOptionValue("s"));
-    }
-    if (cmd.hasOption("q")) {
-      maxCallQueueSize = Integer.parseInt(cmd.getOptionValue("q"));
-    }
-
-    // check for user-defined info server port setting, if so override the conf
-    try {
-      if (cmd.hasOption("infoport")) {
-        String val = cmd.getOptionValue("infoport");
-        conf.setInt("hbase.thrift.info.port", Integer.parseInt(val));
-        log.debug("Web UI port set to " + val);
-      }
-    } catch (NumberFormatException e) {
-      log.error("Could not parse the value provided for the infoport option", e);
-      printUsage();
-      System.exit(1);
-    }
-
-    // Put up info server.
-    startInfoServer(conf);
-
-    final TServer tserver = getServer(workerThreads, selectorThreads, maxCallQueueSize, readTimeout,
-            backlog, nonblocking, hsha, selector, metrics, protocolFactory, processor,
-            transportFactory, inetSocketAddress);
-
-    realUser.doAs(
-      new PrivilegedAction<Object>() {
-        @Override
-        public Object run() {
-          pauseMonitor.start();
-          try {
-            tserver.serve();
-            return null;
-          } finally {
-            pauseMonitor.stop();
-          }
-        }
-      });
-    // when tserver.stop eventually happens we'll get here.
-    return 0;
-  }
-
-  private String getImplType(boolean nonblocking, boolean hsha, boolean selector) {
-    String implType = "threadpool";
-
-    if (nonblocking) {
-      implType = "nonblocking";
-    } else if (hsha) {
-      implType = "hsha";
-    } else if (selector) {
-      implType = "selector";
-    }
-
-    return implType;
-  }
-
-  private String getBindAddress(Configuration conf, CommandLine cmd) {
-    String bindAddress;
-    if (cmd.hasOption("bind")) {
-      bindAddress = cmd.getOptionValue("bind");
-      conf.set("hbase.thrift.info.bindAddress", bindAddress);
-    } else {
-      bindAddress = conf.get("hbase.thrift.info.bindAddress");
-    }
-    return bindAddress;
-  }
-
-  private int getListenPort(Configuration conf, CommandLine cmd) {
-    int listenPort;
-    try {
-      if (cmd.hasOption("port")) {
-        listenPort = Integer.parseInt(cmd.getOptionValue("port"));
-      } else {
-        listenPort = conf.getInt("hbase.regionserver.thrift.port", DEFAULT_LISTEN_PORT);
-      }
-    } catch (NumberFormatException e) {
-      throw new RuntimeException("Could not parse the value provided for the port option", e);
-    }
-    return listenPort;
-  }
-
-  private int getReadTimeout(Configuration conf, CommandLine cmd) {
-    int readTimeout;
-    if (cmd.hasOption(READ_TIMEOUT_OPTION)) {
-      try {
-        readTimeout = Integer.parseInt(cmd.getOptionValue(READ_TIMEOUT_OPTION));
-      } catch (NumberFormatException e) {
-        throw new RuntimeException("Could not parse the value provided for the timeout option", e);
-      }
-    } else {
-      readTimeout = conf.getInt(THRIFT_SERVER_SOCKET_READ_TIMEOUT_KEY,
-        THRIFT_SERVER_SOCKET_READ_TIMEOUT_DEFAULT);
-    }
-    return readTimeout;
-  }
-
-  private void startInfoServer(Configuration conf) throws IOException {
-    int port = conf.getInt("hbase.thrift.info.port", 9095);
-
-    if (port >= 0) {
-      conf.setLong("startcode", System.currentTimeMillis());
-      String a = conf.get("hbase.thrift.info.bindAddress", "0.0.0.0");
-      InfoServer infoServer = new InfoServer("thrift", a, port, false, conf);
-      infoServer.setAttribute("hbase.conf", conf);
-      infoServer.start();
-    }
-  }
-
-  private TServer getServer(int workerThreads, int selectorThreads, int maxCallQueueSize,
-        int readTimeout, int backlog, boolean nonblocking, boolean hsha, boolean selector,
-        ThriftMetrics metrics, TProtocolFactory protocolFactory, TProcessor processor,
-        TTransportFactory transportFactory, InetSocketAddress inetSocketAddress)
-          throws TTransportException {
-    TServer server;
-
-    if (nonblocking) {
-      server = getTNonBlockingServer(protocolFactory, processor, transportFactory,
-              inetSocketAddress);
-    } else if (hsha) {
-      server = getTHsHaServer(protocolFactory, processor, transportFactory, workerThreads,
-              maxCallQueueSize, inetSocketAddress, metrics);
-    } else if (selector) {
-      server = getTThreadedSelectorServer(protocolFactory, processor, transportFactory,
-              workerThreads, selectorThreads, maxCallQueueSize, inetSocketAddress, metrics);
-    } else {
-      server = getTThreadPoolServer(protocolFactory, processor, transportFactory, workerThreads,
-              inetSocketAddress, backlog, readTimeout, metrics);
-    }
-    return server;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b6b4af/hbase-thrift/src/main/resources/hbase-webapps/thrift/thrift.jsp
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/resources/hbase-webapps/thrift/thrift.jsp b/hbase-thrift/src/main/resources/hbase-webapps/thrift/thrift.jsp
index eb329fe..b0064b4 100644
--- a/hbase-thrift/src/main/resources/hbase-webapps/thrift/thrift.jsp
+++ b/hbase-thrift/src/main/resources/hbase-webapps/thrift/thrift.jsp
@@ -20,10 +20,10 @@
 <%@ page contentType="text/html;charset=UTF-8"
   import="org.apache.hadoop.conf.Configuration"
   import="org.apache.hadoop.hbase.HBaseConfiguration"
-  import="org.apache.hadoop.hbase.thrift.ThriftServerRunner.ImplType"
   import="org.apache.hadoop.hbase.util.VersionInfo"
   import="java.util.Date"
 %>
+<%@ page import="org.apache.hadoop.hbase.thrift.ImplType" %>
 
 <%
 Configuration conf = (Configuration)getServletContext().getAttribute("hbase.conf");

http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b6b4af/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftHttpServer.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftHttpServer.java b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftHttpServer.java
index 0eb7641..fe6655e 100644
--- a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftHttpServer.java
+++ b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftHttpServer.java
@@ -17,8 +17,9 @@
  */
 package org.apache.hadoop.hbase.thrift;
 
+import static org.apache.hadoop.hbase.thrift.Constants.INFOPORT_OPTION;
+import static org.apache.hadoop.hbase.thrift.Constants.PORT_OPTION;
 import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNull;
 import static org.junit.Assert.fail;
 
 import java.net.HttpURLConnection;
@@ -65,7 +66,7 @@ public class TestThriftHttpServer {
   private static final Logger LOG =
       LoggerFactory.getLogger(TestThriftHttpServer.class);
 
-  static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  protected static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 
   private Thread httpServerThread;
   private volatile Exception httpServerException;
@@ -97,18 +98,17 @@ public class TestThriftHttpServer {
     conf.set("hbase.thrift.security.qop", "privacy");
     conf.setBoolean("hbase.thrift.ssl.enabled", false);
 
-    ThriftServerRunner runner = null;
+    ThriftServer server = null;
     ExpectedException thrown = ExpectedException.none();
     try {
       thrown.expect(IllegalArgumentException.class);
       thrown.expectMessage("Thrift HTTP Server's QoP is privacy, " +
           "but hbase.thrift.ssl.enabled is false");
-      runner = new ThriftServerRunner(conf);
+      server = new ThriftServer(conf);
+      server.run();
       fail("Thrift HTTP Server starts up even with wrong security configurations.");
     } catch (Exception e) {
     }
-
-    assertNull(runner);
   }
 
   private void startHttpServerThread(final String[] args) {
@@ -117,7 +117,7 @@ public class TestThriftHttpServer {
     httpServerException = null;
     httpServerThread = new Thread(() -> {
       try {
-        thriftServer.doMain(args);
+        thriftServer.run(args);
       } catch (Exception e) {
         httpServerException = e;
       }
@@ -145,6 +145,10 @@ public class TestThriftHttpServer {
     runThriftServer(1024 * 64);
   }
 
+  protected ThriftServer createThriftServer() {
+    return new ThriftServer(TEST_UTIL.getConfiguration());
+  }
+
   @Test
   public void testRunThriftServer() throws Exception {
     runThriftServer(0);
@@ -153,14 +157,14 @@ public class TestThriftHttpServer {
   void runThriftServer(int customHeaderSize) throws Exception {
     List<String> args = new ArrayList<>(3);
     port = HBaseTestingUtility.randomFreePort();
-    args.add("-" + ThriftServer.PORT_OPTION);
+    args.add("-" + PORT_OPTION);
     args.add(String.valueOf(port));
-    args.add("-" + ThriftServer.INFOPORT_OPTION);
+    args.add("-" + INFOPORT_OPTION);
     int infoPort = HBaseTestingUtility.randomFreePort();
     args.add(String.valueOf(infoPort));
     args.add("start");
 
-    thriftServer = new ThriftServer(TEST_UTIL.getConfiguration());
+    thriftServer = createThriftServer();
     startHttpServerThread(args.toArray(new String[args.size()]));
 
     // wait up to 10s for the server to start
@@ -195,9 +199,9 @@ public class TestThriftHttpServer {
     Assert.assertEquals(HttpURLConnection.HTTP_FORBIDDEN, conn.getResponseCode());
   }
 
-  static volatile boolean tableCreated = false;
+  protected static volatile boolean tableCreated = false;
 
-  void talkToThriftServer(String url, int customHeaderSize) throws Exception {
+  protected void talkToThriftServer(String url, int customHeaderSize) throws Exception {
     THttpClient httpClient = new THttpClient(url);
     httpClient.open();
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b6b4af/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServer.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServer.java b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServer.java
index 05dc2ae..09d0605 100644
--- a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServer.java
+++ b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServer.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hbase.thrift;
 
+import static org.apache.hadoop.hbase.thrift.Constants.COALESCE_INC_KEY;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -46,7 +47,6 @@ import org.apache.hadoop.hbase.security.UserProvider;
 import org.apache.hadoop.hbase.test.MetricsAssertHelper;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
-import org.apache.hadoop.hbase.thrift.ThriftServerRunner.HBaseHandler;
 import org.apache.hadoop.hbase.thrift.generated.BatchMutation;
 import org.apache.hadoop.hbase.thrift.generated.ColumnDescriptor;
 import org.apache.hadoop.hbase.thrift.generated.Hbase;
@@ -71,7 +71,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * Unit testing for ThriftServerRunner.HBaseHandler, a part of the
+ * Unit testing for ThriftServerRunner.HBaseServiceHandler, a part of the
  * org.apache.hadoop.hbase.thrift package.
  */
 @Category({ClientTests.class, LargeTests.class})
@@ -113,7 +113,7 @@ public class TestThriftServer {
 
   @BeforeClass
   public static void beforeClass() throws Exception {
-    UTIL.getConfiguration().setBoolean(ThriftServerRunner.COALESCE_INC_KEY, true);
+    UTIL.getConfiguration().setBoolean(COALESCE_INC_KEY, true);
     UTIL.getConfiguration().setBoolean("hbase.table.sanity.checks", false);
     UTIL.getConfiguration().setInt("hbase.client.retries.number", 3);
     UTIL.startMiniCluster();
@@ -152,8 +152,8 @@ public class TestThriftServer {
    * IllegalArgument exception.
    */
   public void doTestTableCreateDrop() throws Exception {
-    ThriftServerRunner.HBaseHandler handler =
-      new ThriftServerRunner.HBaseHandler(UTIL.getConfiguration(),
+    ThriftHBaseServiceHandler handler =
+      new ThriftHBaseServiceHandler(UTIL.getConfiguration(),
         UserProvider.instantiate(UTIL.getConfiguration()));
     doTestTableCreateDrop(handler);
   }
@@ -163,7 +163,7 @@ public class TestThriftServer {
     dropTestTables(handler);
   }
 
-  public static final class MySlowHBaseHandler extends ThriftServerRunner.HBaseHandler
+  public static final class MySlowHBaseHandler extends ThriftHBaseServiceHandler
       implements Hbase.Iface {
 
     protected MySlowHBaseHandler(Configuration c)
@@ -230,7 +230,7 @@ public class TestThriftServer {
   private static Hbase.Iface getHandlerForMetricsTest(ThriftMetrics metrics, Configuration conf)
       throws Exception {
     Hbase.Iface handler = new MySlowHBaseHandler(conf);
-    return HbaseHandlerMetricsProxy.newInstance(handler, metrics, conf);
+    return HbaseHandlerMetricsProxy.newInstance((ThriftHBaseServiceHandler)handler, metrics, conf);
   }
 
   private static ThriftMetrics getMetrics(Configuration conf) throws Exception {
@@ -270,15 +270,15 @@ public class TestThriftServer {
   }
 
   public void doTestIncrements() throws Exception {
-    ThriftServerRunner.HBaseHandler handler =
-      new ThriftServerRunner.HBaseHandler(UTIL.getConfiguration(),
+    ThriftHBaseServiceHandler handler =
+      new ThriftHBaseServiceHandler(UTIL.getConfiguration(),
         UserProvider.instantiate(UTIL.getConfiguration()));
     createTestTables(handler);
     doTestIncrements(handler);
     dropTestTables(handler);
   }
 
-  public static void doTestIncrements(HBaseHandler handler) throws Exception {
+  public static void doTestIncrements(ThriftHBaseServiceHandler handler) throws Exception {
     List<Mutation> mutations = new ArrayList<>(1);
     mutations.add(new Mutation(false, columnAAname, valueEname, true));
     mutations.add(new Mutation(false, columnAname, valueEname, true));
@@ -318,8 +318,8 @@ public class TestThriftServer {
    * versions.
    */
   public void doTestTableMutations() throws Exception {
-    ThriftServerRunner.HBaseHandler handler =
-      new ThriftServerRunner.HBaseHandler(UTIL.getConfiguration(),
+    ThriftHBaseServiceHandler handler =
+      new ThriftHBaseServiceHandler(UTIL.getConfiguration(),
         UserProvider.instantiate(UTIL.getConfiguration()));
     doTestTableMutations(handler);
   }
@@ -395,8 +395,8 @@ public class TestThriftServer {
    */
   public void doTestTableTimestampsAndColumns() throws Exception {
     // Setup
-    ThriftServerRunner.HBaseHandler handler =
-      new ThriftServerRunner.HBaseHandler(UTIL.getConfiguration(),
+    ThriftHBaseServiceHandler handler =
+      new ThriftHBaseServiceHandler(UTIL.getConfiguration(),
         UserProvider.instantiate(UTIL.getConfiguration()));
     handler.createTable(tableAname, getColumnDescriptors());
 
@@ -473,8 +473,8 @@ public class TestThriftServer {
    */
   public void doTestTableScanners() throws Exception {
     // Setup
-    ThriftServerRunner.HBaseHandler handler =
-      new ThriftServerRunner.HBaseHandler(UTIL.getConfiguration(),
+    ThriftHBaseServiceHandler handler =
+      new ThriftHBaseServiceHandler(UTIL.getConfiguration(),
         UserProvider.instantiate(UTIL.getConfiguration()));
     handler.createTable(tableAname, getColumnDescriptors());
 
@@ -592,8 +592,8 @@ public class TestThriftServer {
    * Tests for GetTableRegions
    */
   public void doTestGetTableRegions() throws Exception {
-    ThriftServerRunner.HBaseHandler handler =
-      new ThriftServerRunner.HBaseHandler(UTIL.getConfiguration(),
+    ThriftHBaseServiceHandler handler =
+      new ThriftHBaseServiceHandler(UTIL.getConfiguration(),
         UserProvider.instantiate(UTIL.getConfiguration()));
     doTestGetTableRegions(handler);
   }
@@ -620,7 +620,7 @@ public class TestThriftServer {
 
     conf.set("hbase.thrift.filters", "MyFilter:filterclass");
 
-    ThriftServerRunner.registerFilters(conf);
+    ThriftServer.registerFilters(conf);
 
     Map<String, String> registeredFilters = ParseFilter.getAllFilters();
 
@@ -628,8 +628,8 @@ public class TestThriftServer {
   }
 
   public void doTestGetRegionInfo() throws Exception {
-    ThriftServerRunner.HBaseHandler handler =
-      new ThriftServerRunner.HBaseHandler(UTIL.getConfiguration(),
+    ThriftHBaseServiceHandler handler =
+      new ThriftHBaseServiceHandler(UTIL.getConfiguration(),
         UserProvider.instantiate(UTIL.getConfiguration()));
     doTestGetRegionInfo(handler);
   }
@@ -655,8 +655,8 @@ public class TestThriftServer {
    * Appends the value to a cell and checks that the cell value is updated properly.
    */
   public static void doTestAppend() throws Exception {
-    ThriftServerRunner.HBaseHandler handler =
-      new ThriftServerRunner.HBaseHandler(UTIL.getConfiguration(),
+    ThriftHBaseServiceHandler handler =
+      new ThriftHBaseServiceHandler(UTIL.getConfiguration(),
         UserProvider.instantiate(UTIL.getConfiguration()));
     handler.createTable(tableAname, getColumnDescriptors());
     try {
@@ -687,8 +687,8 @@ public class TestThriftServer {
    * the checkAndPut succeeds.
    */
   public static void doTestCheckAndPut() throws Exception {
-    ThriftServerRunner.HBaseHandler handler =
-      new ThriftServerRunner.HBaseHandler(UTIL.getConfiguration(),
+    ThriftHBaseServiceHandler handler =
+      new ThriftHBaseServiceHandler(UTIL.getConfiguration(),
         UserProvider.instantiate(UTIL.getConfiguration()));
     handler.createTable(tableAname, getColumnDescriptors());
     try {
@@ -729,8 +729,8 @@ public class TestThriftServer {
 
     Configuration conf = UTIL.getConfiguration();
     ThriftMetrics metrics = getMetrics(conf);
-    ThriftServerRunner.HBaseHandler hbaseHandler =
-        new ThriftServerRunner.HBaseHandler(UTIL.getConfiguration(),
+    ThriftHBaseServiceHandler hbaseHandler =
+        new ThriftHBaseServiceHandler(UTIL.getConfiguration(),
             UserProvider.instantiate(UTIL.getConfiguration()));
     Hbase.Iface handler = HbaseHandlerMetricsProxy.newInstance(hbaseHandler, metrics, conf);
 
@@ -863,10 +863,10 @@ public class TestThriftServer {
    * the scanner.
    *
    * @param scannerId the scanner to close
-   * @param handler the HBaseHandler interfacing to HBase
+   * @param handler the HBaseServiceHandler interfacing to HBase
    */
   private void closeScanner(
-      int scannerId, ThriftServerRunner.HBaseHandler handler) throws Exception {
+      int scannerId, ThriftHBaseServiceHandler handler) throws Exception {
     handler.scannerGet(scannerId);
     handler.scannerClose(scannerId);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b6b4af/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServerCmdLine.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServerCmdLine.java b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServerCmdLine.java
index 9e5a103..08dae61 100644
--- a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServerCmdLine.java
+++ b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServerCmdLine.java
@@ -17,6 +17,11 @@
  */
 package org.apache.hadoop.hbase.thrift;
 
+import static org.apache.hadoop.hbase.thrift.Constants.BIND_OPTION;
+import static org.apache.hadoop.hbase.thrift.Constants.COMPACT_OPTION;
+import static org.apache.hadoop.hbase.thrift.Constants.FRAMED_OPTION;
+import static org.apache.hadoop.hbase.thrift.Constants.INFOPORT_OPTION;
+import static org.apache.hadoop.hbase.thrift.Constants.PORT_OPTION;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
@@ -28,7 +33,6 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
-import org.apache.hadoop.hbase.thrift.ThriftServerRunner.ImplType;
 import org.apache.hadoop.hbase.thrift.generated.Hbase;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper;
@@ -68,12 +72,12 @@ public class TestThriftServerCmdLine {
   private static final Logger LOG =
       LoggerFactory.getLogger(TestThriftServerCmdLine.class);
 
-  private final ImplType implType;
-  private boolean specifyFramed;
-  private boolean specifyBindIP;
-  private boolean specifyCompact;
+  protected final ImplType implType;
+  protected boolean specifyFramed;
+  protected boolean specifyBindIP;
+  protected boolean specifyCompact;
 
-  private static final HBaseTestingUtility TEST_UTIL =
+  protected static final HBaseTestingUtility TEST_UTIL =
       new HBaseTestingUtility();
 
   private Thread cmdLineThread;
@@ -81,8 +85,8 @@ public class TestThriftServerCmdLine {
 
   private Exception clientSideException;
 
-  private ThriftServer thriftServer;
-  private int port;
+  private volatile ThriftServer thriftServer;
+  protected int port;
 
   @Parameters
   public static Collection<Object[]> getParameters() {
@@ -143,8 +147,9 @@ public class TestThriftServerCmdLine {
       @Override
       public void run() {
         try {
-          thriftServer.doMain(args);
+          thriftServer.run(args);
         } catch (Exception e) {
+          LOG.error("Error when start thrift server", e);
           cmdLineException = e;
         }
       }
@@ -154,6 +159,10 @@ public class TestThriftServerCmdLine {
     cmdLineThread.start();
   }
 
+  protected ThriftServer createThriftServer() {
+    return new ThriftServer(TEST_UTIL.getConfiguration());
+  }
+
   @Test
   public void testRunThriftServer() throws Exception {
     List<String> args = new ArrayList<>();
@@ -163,37 +172,37 @@ public class TestThriftServerCmdLine {
       args.add(serverTypeOption);
     }
     port = HBaseTestingUtility.randomFreePort();
-    args.add("-" + ThriftServer.PORT_OPTION);
+    args.add("-" + PORT_OPTION);
     args.add(String.valueOf(port));
-    args.add("-" + ThriftServer.INFOPORT_OPTION);
+    args.add("-" + INFOPORT_OPTION);
     int infoPort = HBaseTestingUtility.randomFreePort();
     args.add(String.valueOf(infoPort));
 
     if (specifyFramed) {
-      args.add("-" + ThriftServer.FRAMED_OPTION);
+      args.add("-" + FRAMED_OPTION);
     }
     if (specifyBindIP) {
-      args.add("-" + ThriftServer.BIND_OPTION);
+      args.add("-" + BIND_OPTION);
       args.add(InetAddress.getLocalHost().getHostName());
     }
     if (specifyCompact) {
-      args.add("-" + ThriftServer.COMPACT_OPTION);
+      args.add("-" + COMPACT_OPTION);
     }
     args.add("start");
 
-    thriftServer = new ThriftServer(TEST_UTIL.getConfiguration());
+    thriftServer = createThriftServer();
     startCmdLineThread(args.toArray(new String[args.size()]));
 
     // wait up to 10s for the server to start
     for (int i = 0; i < 100
-        && (thriftServer.serverRunner == null || thriftServer.serverRunner.tserver == null); i++) {
+        && (thriftServer.tserver == null); i++) {
       Thread.sleep(100);
     }
 
     Class<? extends TServer> expectedClass = implType != null ?
         implType.serverClass : TBoundedThreadPoolServer.class;
     assertEquals(expectedClass,
-                 thriftServer.serverRunner.tserver.getClass());
+                 thriftServer.tserver.getClass());
 
     try {
       talkToThriftServer();
@@ -210,9 +219,9 @@ public class TestThriftServerCmdLine {
     }
   }
 
-  private static volatile boolean tableCreated = false;
+  protected static volatile boolean tableCreated = false;
 
-  private void talkToThriftServer() throws Exception {
+  protected void talkToThriftServer() throws Exception {
     TSocket sock = new TSocket(InetAddress.getLocalHost().getHostName(),
         port);
     TTransport transport = sock;
@@ -228,6 +237,7 @@ public class TestThriftServerCmdLine {
       } else {
         prot = new TBinaryProtocol(transport);
       }
+
       Hbase.Client client = new Hbase.Client(prot);
       if (!tableCreated){
         TestThriftServer.createTestTables(client);

http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b6b4af/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftSpnegoHttpServer.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftSpnegoHttpServer.java b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftSpnegoHttpServer.java
index 49eafec..45abe79 100644
--- a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftSpnegoHttpServer.java
+++ b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftSpnegoHttpServer.java
@@ -18,12 +18,7 @@
  */
 package org.apache.hadoop.hbase.thrift;
 
-import static org.apache.hadoop.hbase.thrift.ThriftServerRunner.THRIFT_KERBEROS_PRINCIPAL_KEY;
-import static org.apache.hadoop.hbase.thrift.ThriftServerRunner.THRIFT_KEYTAB_FILE_KEY;
-import static org.apache.hadoop.hbase.thrift.ThriftServerRunner.THRIFT_SPNEGO_KEYTAB_FILE_KEY;
-import static org.apache.hadoop.hbase.thrift.ThriftServerRunner.THRIFT_SPNEGO_PRINCIPAL_KEY;
-import static org.apache.hadoop.hbase.thrift.ThriftServerRunner.THRIFT_SUPPORT_PROXYUSER_KEY;
-import static org.apache.hadoop.hbase.thrift.ThriftServerRunner.USE_HTTP_CONF_KEY;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_SUPPORT_PROXYUSER_KEY;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
@@ -133,14 +128,14 @@ public class TestThriftSpnegoHttpServer extends TestThriftHttpServer {
     HBaseKerberosUtils.setSecuredConfiguration(conf, serverPrincipal, spnegoServerPrincipal);
 
     conf.setBoolean(THRIFT_SUPPORT_PROXYUSER_KEY, true);
-    conf.setBoolean(USE_HTTP_CONF_KEY, true);
+    conf.setBoolean(Constants.USE_HTTP_CONF_KEY, true);
     conf.set("hadoop.proxyuser.hbase.hosts", "*");
     conf.set("hadoop.proxyuser.hbase.groups", "*");
 
-    conf.set(THRIFT_KERBEROS_PRINCIPAL_KEY, serverPrincipal);
-    conf.set(THRIFT_KEYTAB_FILE_KEY, serverKeytab.getAbsolutePath());
-    conf.set(THRIFT_SPNEGO_PRINCIPAL_KEY, spnegoServerPrincipal);
-    conf.set(THRIFT_SPNEGO_KEYTAB_FILE_KEY, spnegoServerKeytab.getAbsolutePath());
+    conf.set(Constants.THRIFT_KERBEROS_PRINCIPAL_KEY, serverPrincipal);
+    conf.set(Constants.THRIFT_KEYTAB_FILE_KEY, serverKeytab.getAbsolutePath());
+    conf.set(Constants.THRIFT_SPNEGO_PRINCIPAL_KEY, spnegoServerPrincipal);
+    conf.set(Constants.THRIFT_SPNEGO_KEYTAB_FILE_KEY, spnegoServerKeytab.getAbsolutePath());
   }
 
   @BeforeClass
@@ -170,7 +165,7 @@ public class TestThriftSpnegoHttpServer extends TestThriftHttpServer {
     spnegoServerKeytab = new File(keytabDir, spnegoServerPrincipal.replace('/', '_') + ".keytab");
     setupUser(kdc, spnegoServerKeytab, spnegoServerPrincipal);
 
-    TEST_UTIL.getConfiguration().setBoolean(USE_HTTP_CONF_KEY, true);
+    TEST_UTIL.getConfiguration().setBoolean(Constants.USE_HTTP_CONF_KEY, true);
     TEST_UTIL.getConfiguration().setBoolean("hbase.table.sanity.checks", false);
     addSecurityConfigurations(TEST_UTIL.getConfiguration());
 
@@ -191,7 +186,7 @@ public class TestThriftSpnegoHttpServer extends TestThriftHttpServer {
   }
 
   @Override
-  void talkToThriftServer(String url, int customHeaderSize) throws Exception {
+  protected void talkToThriftServer(String url, int customHeaderSize) throws Exception {
     // Close httpClient and THttpClient automatically on any failures
     try (
         CloseableHttpClient httpClient = createHttpClient();

http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b6b4af/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThrift2HttpServer.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThrift2HttpServer.java b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThrift2HttpServer.java
new file mode 100644
index 0000000..cf084c9
--- /dev/null
+++ b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThrift2HttpServer.java
@@ -0,0 +1,90 @@
+/**
+ * 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.thrift2;
+
+import java.util.ArrayList;
+
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.thrift.TestThriftHttpServer;
+import org.apache.hadoop.hbase.thrift2.generated.TColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.thrift2.generated.THBaseService;
+import org.apache.hadoop.hbase.thrift2.generated.TTableDescriptor;
+import org.apache.hadoop.hbase.thrift2.generated.TTableName;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TProtocol;
+import org.apache.thrift.transport.THttpClient;
+import org.junit.Assert;
+import org.junit.ClassRule;
+import org.junit.experimental.categories.Category;
+
+@Category({ ClientTests.class, MediumTests.class})
+public class TestThrift2HttpServer extends TestThriftHttpServer {
+  private static final String TABLENAME = "TestThrift2HttpServerTable";
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestThrift2HttpServer.class);
+
+
+
+  @Override
+  protected ThriftServer createThriftServer() {
+    return new ThriftServer(TEST_UTIL.getConfiguration());
+  }
+
+  @Override
+  protected void talkToThriftServer(String url, int customHeaderSize) throws Exception {
+    THttpClient httpClient = new THttpClient(url);
+    httpClient.open();
+
+    if (customHeaderSize > 0) {
+      StringBuilder sb = new StringBuilder();
+      for (int i = 0; i < customHeaderSize; i++) {
+        sb.append("a");
+      }
+      httpClient.setCustomHeader("User-Agent", sb.toString());
+    }
+
+    try {
+      TProtocol prot;
+      prot = new TBinaryProtocol(httpClient);
+      THBaseService.Client client = new THBaseService.Client(prot);
+      TTableName tTableName = new TTableName();
+      tTableName.setNs(Bytes.toBytes(""));
+      tTableName.setQualifier(Bytes.toBytes(TABLENAME));
+      if (!tableCreated){
+        Assert.assertTrue(!client.tableExists(tTableName));
+        TTableDescriptor tTableDescriptor = new TTableDescriptor();
+        tTableDescriptor.setTableName(tTableName);
+        TColumnFamilyDescriptor columnFamilyDescriptor = new TColumnFamilyDescriptor();
+        columnFamilyDescriptor.setName(Bytes.toBytes(TABLENAME));
+        tTableDescriptor.addToColumns(columnFamilyDescriptor);
+        client.createTable(tTableDescriptor, new ArrayList<>());
+        tableCreated = true;
+      }
+      Assert.assertTrue(client.tableExists(tTableName));
+    } finally {
+      httpClient.close();
+    }
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b6b4af/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThrift2ServerCmdLine.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThrift2ServerCmdLine.java b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThrift2ServerCmdLine.java
new file mode 100644
index 0000000..7489bd7
--- /dev/null
+++ b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThrift2ServerCmdLine.java
@@ -0,0 +1,99 @@
+/**
+ * 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.thrift2;
+
+import java.net.InetAddress;
+import java.util.ArrayList;
+
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.thrift.ImplType;
+import org.apache.hadoop.hbase.thrift.TestThriftServerCmdLine;
+import org.apache.hadoop.hbase.thrift2.generated.TColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.thrift2.generated.THBaseService;
+import org.apache.hadoop.hbase.thrift2.generated.TTableDescriptor;
+import org.apache.hadoop.hbase.thrift2.generated.TTableName;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.protocol.TProtocol;
+import org.apache.thrift.transport.TFramedTransport;
+import org.apache.thrift.transport.TSocket;
+import org.apache.thrift.transport.TTransport;
+import org.junit.Assert;
+import org.junit.ClassRule;
+import org.junit.experimental.categories.Category;
+
+@Category({ ClientTests.class, MediumTests.class})
+public class TestThrift2ServerCmdLine extends TestThriftServerCmdLine {
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestThrift2ServerCmdLine.class);
+
+  private static final String TABLENAME = "TestThrift2ServerCmdLineTable";
+
+
+  @Override
+  protected ThriftServer createThriftServer() {
+    return new ThriftServer(TEST_UTIL.getConfiguration());
+  }
+
+  public TestThrift2ServerCmdLine(ImplType implType, boolean specifyFramed,
+      boolean specifyBindIP, boolean specifyCompact) {
+    super(implType, specifyFramed, specifyBindIP, specifyCompact);
+  }
+
+  @Override
+  protected void talkToThriftServer() throws Exception {
+    TSocket sock = new TSocket(InetAddress.getLocalHost().getHostName(),
+        port);
+    TTransport transport = sock;
+    if (specifyFramed || implType.isAlwaysFramed()) {
+      transport = new TFramedTransport(transport);
+    }
+
+    sock.open();
+    try {
+      TProtocol tProtocol;
+      if (specifyCompact) {
+        tProtocol = new TCompactProtocol(transport);
+      } else {
+        tProtocol = new TBinaryProtocol(transport);
+      }
+      THBaseService.Client client = new THBaseService.Client(tProtocol);
+      TTableName tTableName = new TTableName();
+      tTableName.setNs(Bytes.toBytes(""));
+      tTableName.setQualifier(Bytes.toBytes(TABLENAME));
+      if (!tableCreated){
+        Assert.assertTrue(!client.tableExists(tTableName));
+        TTableDescriptor tTableDescriptor = new TTableDescriptor();
+        tTableDescriptor.setTableName(tTableName);
+        TColumnFamilyDescriptor columnFamilyDescriptor = new TColumnFamilyDescriptor();
+        columnFamilyDescriptor.setName(Bytes.toBytes(TABLENAME));
+        tTableDescriptor.addToColumns(columnFamilyDescriptor);
+        client.createTable(tTableDescriptor, new ArrayList<>());
+        tableCreated = true;
+      }
+      Assert.assertTrue(client.tableExists(tTableName));
+    } finally {
+      sock.close();
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b6b4af/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java
index 0734f85..844c1c0 100644
--- a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java
+++ b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java
@@ -18,6 +18,8 @@
 package org.apache.hadoop.hbase.thrift2;
 
 import static java.nio.ByteBuffer.wrap;
+import static org.apache.hadoop.hbase.thrift.HBaseServiceHandler.CLEANUP_INTERVAL;
+import static org.apache.hadoop.hbase.thrift.HBaseServiceHandler.MAX_IDLETIME;
 import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.deleteFromThrift;
 import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.getFromThrift;
 import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.incrementFromThrift;
@@ -71,6 +73,7 @@ import org.apache.hadoop.hbase.test.MetricsAssertHelper;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.thrift.ErrorThrowingGetObserver;
+import org.apache.hadoop.hbase.thrift.HbaseHandlerMetricsProxy;
 import org.apache.hadoop.hbase.thrift.ThriftMetrics;
 import org.apache.hadoop.hbase.thrift2.generated.TAppend;
 import org.apache.hadoop.hbase.thrift2.generated.TColumn;
@@ -115,7 +118,7 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
 
 /**
- * Unit testing for ThriftServer.HBaseHandler, a part of the org.apache.hadoop.hbase.thrift2
+ * Unit testing for ThriftServer.HBaseServiceHandler, a part of the org.apache.hadoop.hbase.thrift2
  * package.
  */
 @Category({ClientTests.class, MediumTests.class})
@@ -749,8 +752,8 @@ public class TestThriftHBaseServiceHandler {
     int cleanUpInterval = 100;
     Configuration conf = new Configuration(UTIL.getConfiguration());
     // Set the ConnectionCache timeout to trigger halfway through the trials
-    conf.setInt(ThriftHBaseServiceHandler.MAX_IDLETIME, (numTrials / 2) * trialPause);
-    conf.setInt(ThriftHBaseServiceHandler.CLEANUP_INTERVAL, cleanUpInterval);
+    conf.setInt(MAX_IDLETIME, (numTrials / 2) * trialPause);
+    conf.setInt(CLEANUP_INTERVAL, cleanUpInterval);
     ThriftHBaseServiceHandler handler = new ThriftHBaseServiceHandler(conf,
         UserProvider.instantiate(conf));
 
@@ -1206,7 +1209,7 @@ public class TestThriftHBaseServiceHandler {
     ThriftMetrics metrics = getMetrics(conf);
     ThriftHBaseServiceHandler hbaseHandler = createHandler();
     THBaseService.Iface handler =
-        ThriftHBaseServiceHandler.newInstance(hbaseHandler, metrics);
+        HbaseHandlerMetricsProxy.newInstance(hbaseHandler, metrics,  conf);
     byte[] rowName = Bytes.toBytes("testMetrics");
     ByteBuffer table = wrap(tableAname);
 
@@ -1249,7 +1252,7 @@ public class TestThriftHBaseServiceHandler {
     ThriftHBaseServiceHandler hbaseHandler = createHandler();
     ThriftMetrics metrics = getMetrics(UTIL.getConfiguration());
     THBaseService.Iface handler =
-        ThriftHBaseServiceHandler.newInstance(hbaseHandler, metrics);
+        HbaseHandlerMetricsProxy.newInstance(hbaseHandler, metrics, null);
     ByteBuffer tTableName = wrap(tableName.getName());
 
     // check metrics increment with a successful get
@@ -1323,7 +1326,7 @@ public class TestThriftHBaseServiceHandler {
       ThriftHBaseServiceHandler hbaseHandler = createHandler();
       ThriftMetrics metrics = getMetrics(UTIL.getConfiguration());
       THBaseService.Iface handler =
-          ThriftHBaseServiceHandler.newInstance(hbaseHandler, metrics);
+          HbaseHandlerMetricsProxy.newInstance(hbaseHandler, metrics, null);
       ByteBuffer tTableName = wrap(tableName.getName());
 
       // check metrics latency with a successful get


[03/15] hbase git commit: HBASE-21652 Refactor ThriftServer making thrift2 server inherited from thrift1 server

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b6b4af/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java
deleted file mode 100644
index 5e248f1..0000000
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java
+++ /dev/null
@@ -1,2031 +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.thrift;
-
-import static org.apache.hadoop.hbase.util.Bytes.getBytes;
-
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.net.UnknownHostException;
-import java.nio.ByteBuffer;
-import java.security.PrivilegedAction;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.TreeMap;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-
-import javax.security.auth.callback.Callback;
-import javax.security.auth.callback.UnsupportedCallbackException;
-import javax.security.sasl.AuthorizeCallback;
-import javax.security.sasl.SaslServer;
-
-import org.apache.commons.lang3.ArrayUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Cell.Type;
-import org.apache.hadoop.hbase.CellBuilder;
-import org.apache.hadoop.hbase.CellBuilderFactory;
-import org.apache.hadoop.hbase.CellBuilderType;
-import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionLocation;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.MetaTableAccessor;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.Append;
-import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.Durability;
-import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.Increment;
-import org.apache.hadoop.hbase.client.OperationWithAttributes;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.RegionInfo;
-import org.apache.hadoop.hbase.client.RegionLocator;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.filter.Filter;
-import org.apache.hadoop.hbase.filter.ParseFilter;
-import org.apache.hadoop.hbase.filter.PrefixFilter;
-import org.apache.hadoop.hbase.filter.WhileMatchFilter;
-import org.apache.hadoop.hbase.http.HttpServerUtil;
-import org.apache.hadoop.hbase.log.HBaseMarkers;
-import org.apache.hadoop.hbase.security.SaslUtil;
-import org.apache.hadoop.hbase.security.SaslUtil.QualityOfProtection;
-import org.apache.hadoop.hbase.security.SecurityUtil;
-import org.apache.hadoop.hbase.security.UserProvider;
-import org.apache.hadoop.hbase.thrift.generated.AlreadyExists;
-import org.apache.hadoop.hbase.thrift.generated.BatchMutation;
-import org.apache.hadoop.hbase.thrift.generated.ColumnDescriptor;
-import org.apache.hadoop.hbase.thrift.generated.Hbase;
-import org.apache.hadoop.hbase.thrift.generated.IOError;
-import org.apache.hadoop.hbase.thrift.generated.IllegalArgument;
-import org.apache.hadoop.hbase.thrift.generated.Mutation;
-import org.apache.hadoop.hbase.thrift.generated.TAppend;
-import org.apache.hadoop.hbase.thrift.generated.TCell;
-import org.apache.hadoop.hbase.thrift.generated.TIncrement;
-import org.apache.hadoop.hbase.thrift.generated.TRegionInfo;
-import org.apache.hadoop.hbase.thrift.generated.TRowResult;
-import org.apache.hadoop.hbase.thrift.generated.TScan;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.ConnectionCache;
-import org.apache.hadoop.hbase.util.DNS;
-import org.apache.hadoop.hbase.util.JvmPauseMonitor;
-import org.apache.hadoop.hbase.util.Strings;
-import org.apache.hadoop.security.SaslRpcServer.SaslGssCallbackHandler;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.authorize.ProxyUsers;
-import org.apache.thrift.TException;
-import org.apache.thrift.TProcessor;
-import org.apache.thrift.protocol.TBinaryProtocol;
-import org.apache.thrift.protocol.TCompactProtocol;
-import org.apache.thrift.protocol.TProtocolFactory;
-import org.apache.thrift.server.THsHaServer;
-import org.apache.thrift.server.TNonblockingServer;
-import org.apache.thrift.server.TServer;
-import org.apache.thrift.server.TServlet;
-import org.apache.thrift.server.TThreadedSelectorServer;
-import org.apache.thrift.transport.TFramedTransport;
-import org.apache.thrift.transport.TNonblockingServerSocket;
-import org.apache.thrift.transport.TNonblockingServerTransport;
-import org.apache.thrift.transport.TSaslServerTransport;
-import org.apache.thrift.transport.TServerSocket;
-import org.apache.thrift.transport.TServerTransport;
-import org.apache.thrift.transport.TTransportFactory;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.eclipse.jetty.http.HttpVersion;
-import org.eclipse.jetty.server.HttpConfiguration;
-import org.eclipse.jetty.server.HttpConnectionFactory;
-import org.eclipse.jetty.server.SecureRequestCustomizer;
-import org.eclipse.jetty.server.Server;
-import org.eclipse.jetty.server.ServerConnector;
-import org.eclipse.jetty.server.SslConnectionFactory;
-import org.eclipse.jetty.servlet.ServletContextHandler;
-import org.eclipse.jetty.servlet.ServletHolder;
-import org.eclipse.jetty.util.ssl.SslContextFactory;
-import org.eclipse.jetty.util.thread.QueuedThreadPool;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.hbase.thirdparty.com.google.common.base.Joiner;
-import org.apache.hbase.thirdparty.com.google.common.base.Splitter;
-import org.apache.hbase.thirdparty.com.google.common.base.Throwables;
-import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
-import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
-import org.apache.hbase.thirdparty.org.apache.commons.cli.Option;
-import org.apache.hbase.thirdparty.org.apache.commons.cli.OptionGroup;
-
-/**
- * ThriftServerRunner - this class starts up a Thrift server which implements
- * the Hbase API specified in the Hbase.thrift IDL file.
- */
-@InterfaceAudience.Private
-public class ThriftServerRunner implements Runnable {
-
-  private static final Logger LOG = LoggerFactory.getLogger(ThriftServerRunner.class);
-
-  private static final int DEFAULT_HTTP_MAX_HEADER_SIZE = 64 * 1024; // 64k
-
-  static final String SERVER_TYPE_CONF_KEY =
-      "hbase.regionserver.thrift.server.type";
-
-  static final String BIND_CONF_KEY = "hbase.regionserver.thrift.ipaddress";
-  static final String COMPACT_CONF_KEY = "hbase.regionserver.thrift.compact";
-  static final String FRAMED_CONF_KEY = "hbase.regionserver.thrift.framed";
-  static final String MAX_FRAME_SIZE_CONF_KEY =
-          "hbase.regionserver.thrift.framed.max_frame_size_in_mb";
-  static final String PORT_CONF_KEY = "hbase.regionserver.thrift.port";
-  static final String COALESCE_INC_KEY = "hbase.regionserver.thrift.coalesceIncrement";
-  static final String USE_HTTP_CONF_KEY = "hbase.regionserver.thrift.http";
-  static final String HTTP_MIN_THREADS_KEY = "hbase.thrift.http_threads.min";
-  static final String HTTP_MAX_THREADS_KEY = "hbase.thrift.http_threads.max";
-
-  static final String THRIFT_SSL_ENABLED_KEY = "hbase.thrift.ssl.enabled";
-  static final String THRIFT_SSL_KEYSTORE_STORE_KEY = "hbase.thrift.ssl.keystore.store";
-  static final String THRIFT_SSL_KEYSTORE_PASSWORD_KEY = "hbase.thrift.ssl.keystore.password";
-  static final String THRIFT_SSL_KEYSTORE_KEYPASSWORD_KEY = "hbase.thrift.ssl.keystore.keypassword";
-  static final String THRIFT_SSL_EXCLUDE_CIPHER_SUITES_KEY =
-      "hbase.thrift.ssl.exclude.cipher.suites";
-  static final String THRIFT_SSL_INCLUDE_CIPHER_SUITES_KEY =
-      "hbase.thrift.ssl.include.cipher.suites";
-  static final String THRIFT_SSL_EXCLUDE_PROTOCOLS_KEY = "hbase.thrift.ssl.exclude.protocols";
-  static final String THRIFT_SSL_INCLUDE_PROTOCOLS_KEY = "hbase.thrift.ssl.include.protocols";
-
-  static final String THRIFT_SUPPORT_PROXYUSER_KEY = "hbase.thrift.support.proxyuser";
-
-  static final String THRIFT_DNS_INTERFACE_KEY = "hbase.thrift.dns.interface";
-  static final String THRIFT_DNS_NAMESERVER_KEY = "hbase.thrift.dns.nameserver";
-  static final String THRIFT_KERBEROS_PRINCIPAL_KEY = "hbase.thrift.kerberos.principal";
-  static final String THRIFT_KEYTAB_FILE_KEY = "hbase.thrift.keytab.file";
-  static final String THRIFT_SPNEGO_PRINCIPAL_KEY = "hbase.thrift.spnego.principal";
-  static final String THRIFT_SPNEGO_KEYTAB_FILE_KEY = "hbase.thrift.spnego.keytab.file";
-
-  /**
-   * Amount of time in milliseconds before a server thread will timeout
-   * waiting for client to send data on a connected socket. Currently,
-   * applies only to TBoundedThreadPoolServer
-   */
-  public static final String THRIFT_SERVER_SOCKET_READ_TIMEOUT_KEY =
-    "hbase.thrift.server.socket.read.timeout";
-  public static final int THRIFT_SERVER_SOCKET_READ_TIMEOUT_DEFAULT = 60000;
-
-
-  /**
-   * Thrift quality of protection configuration key. Valid values can be:
-   * auth-conf: authentication, integrity and confidentiality checking
-   * auth-int: authentication and integrity checking
-   * auth: authentication only
-   *
-   * This is used to authenticate the callers and support impersonation.
-   * The thrift server and the HBase cluster must run in secure mode.
-   */
-  static final String THRIFT_QOP_KEY = "hbase.thrift.security.qop";
-  static final String BACKLOG_CONF_KEY = "hbase.regionserver.thrift.backlog";
-
-  private static final String DEFAULT_BIND_ADDR = "0.0.0.0";
-  public static final int DEFAULT_LISTEN_PORT = 9090;
-  public static final int HREGION_VERSION = 1;
-
-  private final int listenPort;
-
-  private Configuration conf;
-  volatile TServer tserver;
-  volatile Server httpServer;
-  private final Hbase.Iface handler;
-  private final ThriftMetrics metrics;
-  private final HBaseHandler hbaseHandler;
-  private final UserGroupInformation serviceUGI;
-
-  private SaslUtil.QualityOfProtection qop;
-  private String host;
-
-  private final boolean securityEnabled;
-  private final boolean doAsEnabled;
-
-  private final JvmPauseMonitor pauseMonitor;
-
-  static String THRIFT_HTTP_ALLOW_OPTIONS_METHOD = "hbase.thrift.http.allow.options.method";
-  private static boolean THRIFT_HTTP_ALLOW_OPTIONS_METHOD_DEFAULT = false;
-
-  /** An enum of server implementation selections */
-  public enum ImplType {
-    HS_HA("hsha", true, THsHaServer.class, true),
-    NONBLOCKING("nonblocking", true, TNonblockingServer.class, true),
-    THREAD_POOL("threadpool", false, TBoundedThreadPoolServer.class, true),
-    THREADED_SELECTOR("threadedselector", true, TThreadedSelectorServer.class, true);
-
-    public static final ImplType DEFAULT = THREAD_POOL;
-
-    final String option;
-    final boolean isAlwaysFramed;
-    final Class<? extends TServer> serverClass;
-    final boolean canSpecifyBindIP;
-
-    private ImplType(String option, boolean isAlwaysFramed,
-        Class<? extends TServer> serverClass, boolean canSpecifyBindIP) {
-      this.option = option;
-      this.isAlwaysFramed = isAlwaysFramed;
-      this.serverClass = serverClass;
-      this.canSpecifyBindIP = canSpecifyBindIP;
-    }
-
-    /**
-     * @return <code>-option</code>
-     */
-    @Override
-    public String toString() {
-      return "-" + option;
-    }
-
-    public String getOption() {
-      return option;
-    }
-
-    public boolean isAlwaysFramed() {
-      return isAlwaysFramed;
-    }
-
-    public String getDescription() {
-      StringBuilder sb = new StringBuilder("Use the " +
-          serverClass.getSimpleName());
-      if (isAlwaysFramed) {
-        sb.append(" This implies the framed transport.");
-      }
-      if (this == DEFAULT) {
-        sb.append("This is the default.");
-      }
-      return sb.toString();
-    }
-
-    static OptionGroup createOptionGroup() {
-      OptionGroup group = new OptionGroup();
-      for (ImplType t : values()) {
-        group.addOption(new Option(t.option, t.getDescription()));
-      }
-      return group;
-    }
-
-    public static ImplType getServerImpl(Configuration conf) {
-      String confType = conf.get(SERVER_TYPE_CONF_KEY, THREAD_POOL.option);
-      for (ImplType t : values()) {
-        if (confType.equals(t.option)) {
-          return t;
-        }
-      }
-      throw new AssertionError("Unknown server ImplType.option:" + confType);
-    }
-
-    static void setServerImpl(CommandLine cmd, Configuration conf) {
-      ImplType chosenType = null;
-      int numChosen = 0;
-      for (ImplType t : values()) {
-        if (cmd.hasOption(t.option)) {
-          chosenType = t;
-          ++numChosen;
-        }
-      }
-      if (numChosen < 1) {
-        LOG.info("Using default thrift server type");
-        chosenType = DEFAULT;
-      } else if (numChosen > 1) {
-        throw new AssertionError("Exactly one option out of " +
-          Arrays.toString(values()) + " has to be specified");
-      }
-      LOG.info("Using thrift server type " + chosenType.option);
-      conf.set(SERVER_TYPE_CONF_KEY, chosenType.option);
-    }
-
-    public String simpleClassName() {
-      return serverClass.getSimpleName();
-    }
-
-    public static List<String> serversThatCannotSpecifyBindIP() {
-      List<String> l = new ArrayList<>();
-      for (ImplType t : values()) {
-        if (!t.canSpecifyBindIP) {
-          l.add(t.simpleClassName());
-        }
-      }
-      return l;
-    }
-  }
-
-  public ThriftServerRunner(Configuration conf) throws IOException {
-    // login the server principal (if using secure Hadoop)
-    UserProvider userProvider = UserProvider.instantiate(conf);
-    securityEnabled = userProvider.isHadoopSecurityEnabled()
-        && userProvider.isHBaseSecurityEnabled();
-    if (securityEnabled) {
-      host = Strings.domainNamePointerToHostName(DNS.getDefaultHost(
-        conf.get(THRIFT_DNS_INTERFACE_KEY, "default"),
-        conf.get(THRIFT_DNS_NAMESERVER_KEY, "default")));
-      userProvider.login(THRIFT_KEYTAB_FILE_KEY, THRIFT_KERBEROS_PRINCIPAL_KEY, host);
-    }
-    this.serviceUGI = userProvider.getCurrent().getUGI();
-
-    this.conf = HBaseConfiguration.create(conf);
-    this.listenPort = conf.getInt(PORT_CONF_KEY, DEFAULT_LISTEN_PORT);
-    this.metrics = new ThriftMetrics(conf, ThriftMetrics.ThriftServerType.ONE);
-    this.pauseMonitor = new JvmPauseMonitor(conf, this.metrics.getSource());
-    this.hbaseHandler = new HBaseHandler(conf, userProvider);
-    this.hbaseHandler.initMetrics(metrics);
-    this.handler = HbaseHandlerMetricsProxy.newInstance(hbaseHandler, metrics, conf);
-
-    boolean httpEnabled = conf.getBoolean(USE_HTTP_CONF_KEY, false);
-    doAsEnabled = conf.getBoolean(THRIFT_SUPPORT_PROXYUSER_KEY, false);
-    if (doAsEnabled && !httpEnabled) {
-      LOG.warn("Fail to enable the doAs feature. " + USE_HTTP_CONF_KEY + " is not configured");
-    }
-
-    String strQop = conf.get(THRIFT_QOP_KEY);
-    if (strQop != null) {
-      this.qop = SaslUtil.getQop(strQop);
-    }
-    if (qop != null) {
-      if (qop != QualityOfProtection.AUTHENTICATION &&
-          qop != QualityOfProtection.INTEGRITY &&
-          qop != QualityOfProtection.PRIVACY) {
-        throw new IOException(String.format("Invalid %s: It must be one of %s, %s, or %s.",
-                              THRIFT_QOP_KEY,
-                              QualityOfProtection.AUTHENTICATION.name(),
-                              QualityOfProtection.INTEGRITY.name(),
-                              QualityOfProtection.PRIVACY.name()));
-      }
-      checkHttpSecurity(qop, conf);
-      if (!securityEnabled) {
-        throw new IOException("Thrift server must run in secure mode to support authentication");
-      }
-    }
-  }
-
-  private void checkHttpSecurity(QualityOfProtection qop, Configuration conf) {
-    if (qop == QualityOfProtection.PRIVACY &&
-        conf.getBoolean(USE_HTTP_CONF_KEY, false) &&
-        !conf.getBoolean(THRIFT_SSL_ENABLED_KEY, false)) {
-      throw new IllegalArgumentException("Thrift HTTP Server's QoP is privacy, but " +
-          THRIFT_SSL_ENABLED_KEY + " is false");
-    }
-  }
-
-  /*
-   * Runs the Thrift server
-   */
-  @Override
-  public void run() {
-    serviceUGI.doAs(new PrivilegedAction<Object>() {
-      @Override
-      public Object run() {
-        try {
-          pauseMonitor.start();
-          if (conf.getBoolean(USE_HTTP_CONF_KEY, false)) {
-            setupHTTPServer();
-            httpServer.start();
-            httpServer.join();
-          } else {
-            setupServer();
-            tserver.serve();
-          }
-        } catch (Exception e) {
-          LOG.error(HBaseMarkers.FATAL, "Cannot run ThriftServer", e);
-          // Crash the process if the ThriftServer is not running
-          System.exit(-1);
-        }
-        return null;
-      }
-    });
-
-  }
-
-  public void shutdown() {
-    if (pauseMonitor != null) {
-      pauseMonitor.stop();
-    }
-    if (tserver != null) {
-      tserver.stop();
-      tserver = null;
-    }
-    if (httpServer != null) {
-      try {
-        httpServer.stop();
-        httpServer = null;
-      } catch (Exception e) {
-        LOG.error("Problem encountered in shutting down HTTP server", e);
-      }
-      httpServer = null;
-    }
-  }
-
-  private void setupHTTPServer() throws IOException {
-    TProtocolFactory protocolFactory = new TBinaryProtocol.Factory();
-    TProcessor processor = new Hbase.Processor<>(handler);
-    TServlet thriftHttpServlet = new ThriftHttpServlet(processor, protocolFactory, serviceUGI,
-        conf, hbaseHandler, securityEnabled, doAsEnabled);
-
-    // Set the default max thread number to 100 to limit
-    // the number of concurrent requests so that Thrfit HTTP server doesn't OOM easily.
-    // Jetty set the default max thread number to 250, if we don't set it.
-    //
-    // Our default min thread number 2 is the same as that used by Jetty.
-    int minThreads = conf.getInt(HTTP_MIN_THREADS_KEY, 2);
-    int maxThreads = conf.getInt(HTTP_MAX_THREADS_KEY, 100);
-    QueuedThreadPool threadPool = new QueuedThreadPool(maxThreads);
-    threadPool.setMinThreads(minThreads);
-    httpServer = new Server(threadPool);
-
-    // Context handler
-    ServletContextHandler ctxHandler = new ServletContextHandler(httpServer, "/",
-            ServletContextHandler.SESSIONS);
-    ctxHandler.addServlet(new ServletHolder(thriftHttpServlet), "/*");
-    HttpServerUtil.constrainHttpMethods(ctxHandler,
-      conf.getBoolean(THRIFT_HTTP_ALLOW_OPTIONS_METHOD, THRIFT_HTTP_ALLOW_OPTIONS_METHOD_DEFAULT));
-
-    // set up Jetty and run the embedded server
-    HttpConfiguration httpConfig = new HttpConfiguration();
-    httpConfig.setSecureScheme("https");
-    httpConfig.setSecurePort(listenPort);
-    httpConfig.setHeaderCacheSize(DEFAULT_HTTP_MAX_HEADER_SIZE);
-    httpConfig.setRequestHeaderSize(DEFAULT_HTTP_MAX_HEADER_SIZE);
-    httpConfig.setResponseHeaderSize(DEFAULT_HTTP_MAX_HEADER_SIZE);
-    httpConfig.setSendServerVersion(false);
-    httpConfig.setSendDateHeader(false);
-
-    ServerConnector serverConnector;
-    if(conf.getBoolean(THRIFT_SSL_ENABLED_KEY, false)) {
-      HttpConfiguration httpsConfig = new HttpConfiguration(httpConfig);
-      httpsConfig.addCustomizer(new SecureRequestCustomizer());
-
-      SslContextFactory sslCtxFactory = new SslContextFactory();
-      String keystore = conf.get(THRIFT_SSL_KEYSTORE_STORE_KEY);
-      String password = HBaseConfiguration.getPassword(conf,
-          THRIFT_SSL_KEYSTORE_PASSWORD_KEY, null);
-      String keyPassword = HBaseConfiguration.getPassword(conf,
-          THRIFT_SSL_KEYSTORE_KEYPASSWORD_KEY, password);
-      sslCtxFactory.setKeyStorePath(keystore);
-      sslCtxFactory.setKeyStorePassword(password);
-      sslCtxFactory.setKeyManagerPassword(keyPassword);
-
-      String[] excludeCiphers = conf.getStrings(
-          THRIFT_SSL_EXCLUDE_CIPHER_SUITES_KEY, ArrayUtils.EMPTY_STRING_ARRAY);
-      if (excludeCiphers.length != 0) {
-        sslCtxFactory.setExcludeCipherSuites(excludeCiphers);
-      }
-      String[] includeCiphers = conf.getStrings(
-          THRIFT_SSL_INCLUDE_CIPHER_SUITES_KEY, ArrayUtils.EMPTY_STRING_ARRAY);
-      if (includeCiphers.length != 0) {
-        sslCtxFactory.setIncludeCipherSuites(includeCiphers);
-      }
-
-      // Disable SSLv3 by default due to "Poodle" Vulnerability - CVE-2014-3566
-      String[] excludeProtocols = conf.getStrings(
-          THRIFT_SSL_EXCLUDE_PROTOCOLS_KEY, "SSLv3");
-      if (excludeProtocols.length != 0) {
-        sslCtxFactory.setExcludeProtocols(excludeProtocols);
-      }
-      String[] includeProtocols = conf.getStrings(
-          THRIFT_SSL_INCLUDE_PROTOCOLS_KEY, ArrayUtils.EMPTY_STRING_ARRAY);
-      if (includeProtocols.length != 0) {
-        sslCtxFactory.setIncludeProtocols(includeProtocols);
-      }
-
-      serverConnector = new ServerConnector(httpServer,
-          new SslConnectionFactory(sslCtxFactory, HttpVersion.HTTP_1_1.toString()),
-          new HttpConnectionFactory(httpsConfig));
-    } else {
-      serverConnector = new ServerConnector(httpServer, new HttpConnectionFactory(httpConfig));
-    }
-    serverConnector.setPort(listenPort);
-    serverConnector.setHost(getBindAddress(conf).getHostAddress());
-    httpServer.addConnector(serverConnector);
-    httpServer.setStopAtShutdown(true);
-
-    if (doAsEnabled) {
-      ProxyUsers.refreshSuperUserGroupsConfiguration(conf);
-    }
-
-    LOG.info("Starting Thrift HTTP Server on {}", Integer.toString(listenPort));
-  }
-
-  /**
-   * Setting up the thrift TServer
-   */
-  private void setupServer() throws Exception {
-    // Construct correct ProtocolFactory
-    TProtocolFactory protocolFactory = getProtocolFactory();
-
-    final TProcessor p = new Hbase.Processor<>(handler);
-    ImplType implType = ImplType.getServerImpl(conf);
-    TProcessor processor = p;
-
-    // Construct correct TransportFactory
-    TTransportFactory transportFactory;
-    if (conf.getBoolean(FRAMED_CONF_KEY, false) || implType.isAlwaysFramed) {
-      if (qop != null) {
-        throw new RuntimeException("Thrift server authentication"
-          + " doesn't work with framed transport yet");
-      }
-      transportFactory = new TFramedTransport.Factory(
-          conf.getInt(MAX_FRAME_SIZE_CONF_KEY, 2)  * 1024 * 1024);
-      LOG.debug("Using framed transport");
-    } else if (qop == null) {
-      transportFactory = new TTransportFactory();
-    } else {
-      // Extract the name from the principal
-      String thriftKerberosPrincipal = conf.get(THRIFT_KERBEROS_PRINCIPAL_KEY);
-      if (thriftKerberosPrincipal == null) {
-        throw new IllegalArgumentException(THRIFT_KERBEROS_PRINCIPAL_KEY + " cannot be null");
-      }
-      String name = SecurityUtil.getUserFromPrincipal(thriftKerberosPrincipal);
-      Map<String, String> saslProperties = SaslUtil.initSaslProperties(qop.name());
-      TSaslServerTransport.Factory saslFactory = new TSaslServerTransport.Factory();
-      saslFactory.addServerDefinition("GSSAPI", name, host, saslProperties,
-        new SaslGssCallbackHandler() {
-          @Override
-          public void handle(Callback[] callbacks)
-              throws UnsupportedCallbackException {
-            AuthorizeCallback ac = null;
-            for (Callback callback : callbacks) {
-              if (callback instanceof AuthorizeCallback) {
-                ac = (AuthorizeCallback) callback;
-              } else {
-                throw new UnsupportedCallbackException(callback,
-                    "Unrecognized SASL GSSAPI Callback");
-              }
-            }
-            if (ac != null) {
-              String authid = ac.getAuthenticationID();
-              String authzid = ac.getAuthorizationID();
-              if (!authid.equals(authzid)) {
-                ac.setAuthorized(false);
-              } else {
-                ac.setAuthorized(true);
-                String userName = SecurityUtil.getUserFromPrincipal(authzid);
-                LOG.info("Effective user: {}", userName);
-                ac.setAuthorizedID(userName);
-              }
-            }
-          }
-        });
-      transportFactory = saslFactory;
-
-      // Create a processor wrapper, to get the caller
-      processor = (inProt, outProt) -> {
-        TSaslServerTransport saslServerTransport =
-          (TSaslServerTransport)inProt.getTransport();
-        SaslServer saslServer = saslServerTransport.getSaslServer();
-        String principal = saslServer.getAuthorizationID();
-        hbaseHandler.setEffectiveUser(principal);
-        return p.process(inProt, outProt);
-      };
-    }
-
-    if (conf.get(BIND_CONF_KEY) != null && !implType.canSpecifyBindIP) {
-      LOG.error("Server types {} don't support IP address binding at the moment. See " +
-          "https://issues.apache.org/jira/browse/HBASE-2155 for details.",
-          Joiner.on(", ").join(ImplType.serversThatCannotSpecifyBindIP()));
-      throw new RuntimeException("-" + BIND_CONF_KEY + " not supported with " + implType);
-    }
-
-    // Thrift's implementation uses '0' as a placeholder for 'use the default.'
-    int backlog = conf.getInt(BACKLOG_CONF_KEY, 0);
-
-    if (implType == ImplType.HS_HA || implType == ImplType.NONBLOCKING ||
-        implType == ImplType.THREADED_SELECTOR) {
-      InetAddress listenAddress = getBindAddress(conf);
-      TNonblockingServerTransport serverTransport = new TNonblockingServerSocket(
-          new InetSocketAddress(listenAddress, listenPort));
-
-      if (implType == ImplType.NONBLOCKING) {
-        TNonblockingServer.Args serverArgs =
-            new TNonblockingServer.Args(serverTransport);
-        serverArgs.processor(processor)
-                  .transportFactory(transportFactory)
-                  .protocolFactory(protocolFactory);
-        tserver = new TNonblockingServer(serverArgs);
-      } else if (implType == ImplType.HS_HA) {
-        THsHaServer.Args serverArgs = new THsHaServer.Args(serverTransport);
-        CallQueue callQueue = new CallQueue(new LinkedBlockingQueue<>(), metrics);
-        ExecutorService executorService = createExecutor(
-            callQueue, serverArgs.getMaxWorkerThreads(), serverArgs.getMaxWorkerThreads());
-        serverArgs.executorService(executorService).processor(processor)
-                .transportFactory(transportFactory).protocolFactory(protocolFactory);
-        tserver = new THsHaServer(serverArgs);
-      } else { // THREADED_SELECTOR
-        TThreadedSelectorServer.Args serverArgs =
-            new HThreadedSelectorServerArgs(serverTransport, conf);
-        CallQueue callQueue = new CallQueue(new LinkedBlockingQueue<>(), metrics);
-        ExecutorService executorService = createExecutor(
-            callQueue, serverArgs.getWorkerThreads(), serverArgs.getWorkerThreads());
-        serverArgs.executorService(executorService).processor(processor)
-                .transportFactory(transportFactory).protocolFactory(protocolFactory);
-        tserver = new TThreadedSelectorServer(serverArgs);
-      }
-      LOG.info("starting HBase {} server on {}", implType.simpleClassName(),
-          Integer.toString(listenPort));
-    } else if (implType == ImplType.THREAD_POOL) {
-      // Thread pool server. Get the IP address to bind to.
-      InetAddress listenAddress = getBindAddress(conf);
-      int readTimeout = conf.getInt(THRIFT_SERVER_SOCKET_READ_TIMEOUT_KEY,
-          THRIFT_SERVER_SOCKET_READ_TIMEOUT_DEFAULT);
-      TServerTransport serverTransport = new TServerSocket(
-          new TServerSocket.ServerSocketTransportArgs().
-              bindAddr(new InetSocketAddress(listenAddress, listenPort)).backlog(backlog).
-              clientTimeout(readTimeout));
-
-      TBoundedThreadPoolServer.Args serverArgs =
-          new TBoundedThreadPoolServer.Args(serverTransport, conf);
-      serverArgs.processor(processor).transportFactory(transportFactory)
-              .protocolFactory(protocolFactory);
-      LOG.info("starting " + ImplType.THREAD_POOL.simpleClassName() + " on "
-          + listenAddress + ":" + Integer.toString(listenPort)
-          + " with readTimeout " + readTimeout + "ms; " + serverArgs);
-      this.tserver = new TBoundedThreadPoolServer(serverArgs, metrics);
-    } else {
-      throw new AssertionError("Unsupported Thrift server implementation: " +
-          implType.simpleClassName());
-    }
-
-    // A sanity check that we instantiated the right type of server.
-    if (tserver.getClass() != implType.serverClass) {
-      throw new AssertionError("Expected to create Thrift server class " +
-          implType.serverClass.getName() + " but got " +
-          tserver.getClass().getName());
-    }
-
-
-
-    registerFilters(conf);
-  }
-
-  private TProtocolFactory getProtocolFactory() {
-    TProtocolFactory protocolFactory;
-
-    if (conf.getBoolean(COMPACT_CONF_KEY, false)) {
-      LOG.debug("Using compact protocol");
-      protocolFactory = new TCompactProtocol.Factory();
-    } else {
-      LOG.debug("Using binary protocol");
-      protocolFactory = new TBinaryProtocol.Factory();
-    }
-
-    return protocolFactory;
-  }
-
-  ExecutorService createExecutor(BlockingQueue<Runnable> callQueue,
-                                 int minWorkers, int maxWorkers) {
-    ThreadFactoryBuilder tfb = new ThreadFactoryBuilder();
-    tfb.setDaemon(true);
-    tfb.setNameFormat("thrift-worker-%d");
-    ThreadPoolExecutor threadPool = new THBaseThreadPoolExecutor(minWorkers, maxWorkers,
-            Long.MAX_VALUE, TimeUnit.SECONDS, callQueue, tfb.build(), metrics);
-    threadPool.allowCoreThreadTimeOut(true);
-    return threadPool;
-  }
-
-  private InetAddress getBindAddress(Configuration conf)
-      throws UnknownHostException {
-    String bindAddressStr = conf.get(BIND_CONF_KEY, DEFAULT_BIND_ADDR);
-    return InetAddress.getByName(bindAddressStr);
-  }
-
-  protected static class ResultScannerWrapper {
-
-    private final ResultScanner scanner;
-    private final boolean sortColumns;
-    public ResultScannerWrapper(ResultScanner resultScanner,
-                                boolean sortResultColumns) {
-      scanner = resultScanner;
-      sortColumns = sortResultColumns;
-    }
-
-    public ResultScanner getScanner() {
-      return scanner;
-    }
-
-    public boolean isColumnSorted() {
-      return sortColumns;
-    }
-  }
-
-  /**
-   * The HBaseHandler is a glue object that connects Thrift RPC calls to the
-   * HBase client API primarily defined in the Admin and Table objects.
-   */
-  public static class HBaseHandler implements Hbase.Iface {
-    protected Configuration conf;
-    protected static final Logger LOG = LoggerFactory.getLogger(HBaseHandler.class);
-
-    // nextScannerId and scannerMap are used to manage scanner state
-    protected int nextScannerId = 0;
-    protected HashMap<Integer, ResultScannerWrapper> scannerMap;
-    private ThriftMetrics metrics = null;
-
-    private final ConnectionCache connectionCache;
-    IncrementCoalescer coalescer;
-
-    static final String CLEANUP_INTERVAL = "hbase.thrift.connection.cleanup-interval";
-    static final String MAX_IDLETIME = "hbase.thrift.connection.max-idletime";
-
-    /**
-     * Returns a list of all the column families for a given Table.
-     *
-     * @param table
-     * @throws IOException
-     */
-    byte[][] getAllColumns(Table table) throws IOException {
-      HColumnDescriptor[] cds = table.getTableDescriptor().getColumnFamilies();
-      byte[][] columns = new byte[cds.length][];
-      for (int i = 0; i < cds.length; i++) {
-        columns[i] = Bytes.add(cds[i].getName(),
-            KeyValue.COLUMN_FAMILY_DELIM_ARRAY);
-      }
-      return columns;
-    }
-
-    /**
-     * Creates and returns a Table instance from a given table name.
-     *
-     * @param tableName
-     *          name of table
-     * @return Table object
-     * @throws IOException if getting the table fails
-     */
-    public Table getTable(final byte[] tableName) throws IOException {
-      String table = Bytes.toString(tableName);
-      return connectionCache.getTable(table);
-    }
-
-    public Table getTable(final ByteBuffer tableName) throws IOException {
-      return getTable(getBytes(tableName));
-    }
-
-    /**
-     * Assigns a unique ID to the scanner and adds the mapping to an internal
-     * hash-map.
-     *
-     * @param scanner the {@link ResultScanner} to add
-     * @return integer scanner id
-     */
-    protected synchronized int addScanner(ResultScanner scanner, boolean sortColumns) {
-      int id = nextScannerId++;
-      ResultScannerWrapper resultScannerWrapper = new ResultScannerWrapper(scanner, sortColumns);
-      scannerMap.put(id, resultScannerWrapper);
-      return id;
-    }
-
-    /**
-     * Returns the scanner associated with the specified ID.
-     *
-     * @param id the ID of the scanner to get
-     * @return a Scanner, or null if ID was invalid.
-     */
-    protected synchronized ResultScannerWrapper getScanner(int id) {
-      return scannerMap.get(id);
-    }
-
-    /**
-     * Removes the scanner associated with the specified ID from the internal
-     * id-&gt;scanner hash-map.
-     *
-     * @param id the ID of the scanner to remove
-     * @return a Scanner, or null if ID was invalid.
-     */
-    protected synchronized ResultScannerWrapper removeScanner(int id) {
-      return scannerMap.remove(id);
-    }
-
-    protected HBaseHandler(final Configuration c,
-        final UserProvider userProvider) throws IOException {
-      this.conf = c;
-      scannerMap = new HashMap<>();
-      this.coalescer = new IncrementCoalescer(this);
-
-      int cleanInterval = conf.getInt(CLEANUP_INTERVAL, 10 * 1000);
-      int maxIdleTime = conf.getInt(MAX_IDLETIME, 10 * 60 * 1000);
-      connectionCache = new ConnectionCache(
-        conf, userProvider, cleanInterval, maxIdleTime);
-    }
-
-    /**
-     * Obtain HBaseAdmin. Creates the instance if it is not already created.
-     */
-    private Admin getAdmin() throws IOException {
-      return connectionCache.getAdmin();
-    }
-
-    void setEffectiveUser(String effectiveUser) {
-      connectionCache.setEffectiveUser(effectiveUser);
-    }
-
-    @Override
-    public void enableTable(ByteBuffer tableName) throws IOError {
-      try{
-        getAdmin().enableTable(getTableName(tableName));
-      } catch (IOException e) {
-        LOG.warn(e.getMessage(), e);
-        throw getIOError(e);
-      }
-    }
-
-    @Override
-    public void disableTable(ByteBuffer tableName) throws IOError{
-      try{
-        getAdmin().disableTable(getTableName(tableName));
-      } catch (IOException e) {
-        LOG.warn(e.getMessage(), e);
-        throw getIOError(e);
-      }
-    }
-
-    @Override
-    public boolean isTableEnabled(ByteBuffer tableName) throws IOError {
-      try {
-        return this.connectionCache.getAdmin().isTableEnabled(getTableName(tableName));
-      } catch (IOException e) {
-        LOG.warn(e.getMessage(), e);
-        throw getIOError(e);
-      }
-    }
-
-    // ThriftServerRunner.compact should be deprecated and replaced with methods specific to
-    // table and region.
-    @Override
-    public void compact(ByteBuffer tableNameOrRegionName) throws IOError {
-      try {
-        try {
-          getAdmin().compactRegion(getBytes(tableNameOrRegionName));
-        } catch (IllegalArgumentException e) {
-          // Invalid region, try table
-          getAdmin().compact(TableName.valueOf(getBytes(tableNameOrRegionName)));
-        }
-      } catch (IOException e) {
-        LOG.warn(e.getMessage(), e);
-        throw getIOError(e);
-      }
-    }
-
-    // ThriftServerRunner.majorCompact should be deprecated and replaced with methods specific
-    // to table and region.
-    @Override
-    public void majorCompact(ByteBuffer tableNameOrRegionName) throws IOError {
-      try {
-        try {
-          getAdmin().compactRegion(getBytes(tableNameOrRegionName));
-        } catch (IllegalArgumentException e) {
-          // Invalid region, try table
-          getAdmin().compact(TableName.valueOf(getBytes(tableNameOrRegionName)));
-        }
-      } catch (IOException e) {
-        LOG.warn(e.getMessage(), e);
-        throw getIOError(e);
-      }
-    }
-
-    @Override
-    public List<ByteBuffer> getTableNames() throws IOError {
-      try {
-        TableName[] tableNames = this.getAdmin().listTableNames();
-        ArrayList<ByteBuffer> list = new ArrayList<>(tableNames.length);
-        for (TableName tableName : tableNames) {
-          list.add(ByteBuffer.wrap(tableName.getName()));
-        }
-        return list;
-      } catch (IOException e) {
-        LOG.warn(e.getMessage(), e);
-        throw getIOError(e);
-      }
-    }
-
-    /**
-     * @return the list of regions in the given table, or an empty list if the table does not exist
-     */
-    @Override
-    public List<TRegionInfo> getTableRegions(ByteBuffer tableName) throws IOError {
-      try (RegionLocator locator = connectionCache.getRegionLocator(getBytes(tableName))) {
-        List<HRegionLocation> regionLocations = locator.getAllRegionLocations();
-        List<TRegionInfo> results = new ArrayList<>(regionLocations.size());
-        for (HRegionLocation regionLocation : regionLocations) {
-          RegionInfo info = regionLocation.getRegionInfo();
-          ServerName serverName = regionLocation.getServerName();
-          TRegionInfo region = new TRegionInfo();
-          region.serverName = ByteBuffer.wrap(
-              Bytes.toBytes(serverName.getHostname()));
-          region.port = serverName.getPort();
-          region.startKey = ByteBuffer.wrap(info.getStartKey());
-          region.endKey = ByteBuffer.wrap(info.getEndKey());
-          region.id = info.getRegionId();
-          region.name = ByteBuffer.wrap(info.getRegionName());
-          region.version = HREGION_VERSION; // HRegion now not versioned, PB encoding used
-          results.add(region);
-        }
-        return results;
-      } catch (TableNotFoundException e) {
-        // Return empty list for non-existing table
-        return Collections.emptyList();
-      } catch (IOException e){
-        LOG.warn(e.getMessage(), e);
-        throw getIOError(e);
-      }
-    }
-
-    @Override
-    public List<TCell> get(
-        ByteBuffer tableName, ByteBuffer row, ByteBuffer column,
-        Map<ByteBuffer, ByteBuffer> attributes)
-        throws IOError {
-      byte [][] famAndQf = CellUtil.parseColumn(getBytes(column));
-      if (famAndQf.length == 1) {
-        return get(tableName, row, famAndQf[0], null, attributes);
-      }
-      if (famAndQf.length == 2) {
-        return get(tableName, row, famAndQf[0], famAndQf[1], attributes);
-      }
-      throw new IllegalArgumentException("Invalid familyAndQualifier provided.");
-    }
-
-    /**
-     * Note: this internal interface is slightly different from public APIs in regard to handling
-     * of the qualifier. Here we differ from the public Java API in that null != byte[0]. Rather,
-     * we respect qual == null as a request for the entire column family. The caller (
-     * {@link #get(ByteBuffer, ByteBuffer, ByteBuffer, Map)}) interface IS consistent in that the
-     * column is parse like normal.
-     */
-    protected List<TCell> get(ByteBuffer tableName,
-                              ByteBuffer row,
-                              byte[] family,
-                              byte[] qualifier,
-                              Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
-      Table table = null;
-      try {
-        table = getTable(tableName);
-        Get get = new Get(getBytes(row));
-        addAttributes(get, attributes);
-        if (qualifier == null) {
-          get.addFamily(family);
-        } else {
-          get.addColumn(family, qualifier);
-        }
-        Result result = table.get(get);
-        return ThriftUtilities.cellFromHBase(result.rawCells());
-      } catch (IOException e) {
-        LOG.warn(e.getMessage(), e);
-        throw getIOError(e);
-      } finally {
-        closeTable(table);
-      }
-    }
-
-    @Override
-    public List<TCell> getVer(ByteBuffer tableName, ByteBuffer row, ByteBuffer column,
-        int numVersions, Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
-      byte [][] famAndQf = CellUtil.parseColumn(getBytes(column));
-      if(famAndQf.length == 1) {
-        return getVer(tableName, row, famAndQf[0], null, numVersions, attributes);
-      }
-      if (famAndQf.length == 2) {
-        return getVer(tableName, row, famAndQf[0], famAndQf[1], numVersions, attributes);
-      }
-      throw new IllegalArgumentException("Invalid familyAndQualifier provided.");
-
-    }
-
-    /**
-     * Note: this public interface is slightly different from public Java APIs in regard to
-     * handling of the qualifier. Here we differ from the public Java API in that null != byte[0].
-     * Rather, we respect qual == null as a request for the entire column family. If you want to
-     * access the entire column family, use
-     * {@link #getVer(ByteBuffer, ByteBuffer, ByteBuffer, int, Map)} with a {@code column} value
-     * that lacks a {@code ':'}.
-     */
-    public List<TCell> getVer(ByteBuffer tableName, ByteBuffer row, byte[] family,
-        byte[] qualifier, int numVersions, Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
-
-      Table table = null;
-      try {
-        table = getTable(tableName);
-        Get get = new Get(getBytes(row));
-        addAttributes(get, attributes);
-        if (null == qualifier) {
-          get.addFamily(family);
-        } else {
-          get.addColumn(family, qualifier);
-        }
-        get.setMaxVersions(numVersions);
-        Result result = table.get(get);
-        return ThriftUtilities.cellFromHBase(result.rawCells());
-      } catch (IOException e) {
-        LOG.warn(e.getMessage(), e);
-        throw getIOError(e);
-      } finally{
-        closeTable(table);
-      }
-    }
-
-    @Override
-    public List<TCell> getVerTs(ByteBuffer tableName, ByteBuffer row, ByteBuffer column,
-        long timestamp, int numVersions, Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
-      byte [][] famAndQf = CellUtil.parseColumn(getBytes(column));
-      if (famAndQf.length == 1) {
-        return getVerTs(tableName, row, famAndQf[0], null, timestamp, numVersions, attributes);
-      }
-      if (famAndQf.length == 2) {
-        return getVerTs(tableName, row, famAndQf[0], famAndQf[1], timestamp, numVersions,
-          attributes);
-      }
-      throw new IllegalArgumentException("Invalid familyAndQualifier provided.");
-    }
-
-    /**
-     * Note: this internal interface is slightly different from public APIs in regard to handling
-     * of the qualifier. Here we differ from the public Java API in that null != byte[0]. Rather,
-     * we respect qual == null as a request for the entire column family. The caller (
-     * {@link #getVerTs(ByteBuffer, ByteBuffer, ByteBuffer, long, int, Map)}) interface IS
-     * consistent in that the column is parse like normal.
-     */
-    protected List<TCell> getVerTs(ByteBuffer tableName, ByteBuffer row, byte[] family,
-        byte[] qualifier, long timestamp, int numVersions, Map<ByteBuffer, ByteBuffer> attributes)
-        throws IOError {
-
-      Table table = null;
-      try {
-        table = getTable(tableName);
-        Get get = new Get(getBytes(row));
-        addAttributes(get, attributes);
-        if (null == qualifier) {
-          get.addFamily(family);
-        } else {
-          get.addColumn(family, qualifier);
-        }
-        get.setTimeRange(0, timestamp);
-        get.setMaxVersions(numVersions);
-        Result result = table.get(get);
-        return ThriftUtilities.cellFromHBase(result.rawCells());
-      } catch (IOException e) {
-        LOG.warn(e.getMessage(), e);
-        throw getIOError(e);
-      } finally{
-        closeTable(table);
-      }
-    }
-
-    @Override
-    public List<TRowResult> getRow(ByteBuffer tableName, ByteBuffer row,
-        Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
-      return getRowWithColumnsTs(tableName, row, null,
-                                 HConstants.LATEST_TIMESTAMP,
-                                 attributes);
-    }
-
-    @Override
-    public List<TRowResult> getRowWithColumns(ByteBuffer tableName,
-                                              ByteBuffer row,
-        List<ByteBuffer> columns,
-        Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
-      return getRowWithColumnsTs(tableName, row, columns,
-                                 HConstants.LATEST_TIMESTAMP,
-                                 attributes);
-    }
-
-    @Override
-    public List<TRowResult> getRowTs(ByteBuffer tableName, ByteBuffer row,
-        long timestamp, Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
-      return getRowWithColumnsTs(tableName, row, null,
-                                 timestamp, attributes);
-    }
-
-    @Override
-    public List<TRowResult> getRowWithColumnsTs(
-        ByteBuffer tableName, ByteBuffer row, List<ByteBuffer> columns,
-        long timestamp, Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
-
-      Table table = null;
-      try {
-        table = getTable(tableName);
-        if (columns == null) {
-          Get get = new Get(getBytes(row));
-          addAttributes(get, attributes);
-          get.setTimeRange(0, timestamp);
-          Result result = table.get(get);
-          return ThriftUtilities.rowResultFromHBase(result);
-        }
-        Get get = new Get(getBytes(row));
-        addAttributes(get, attributes);
-        for(ByteBuffer column : columns) {
-          byte [][] famAndQf = CellUtil.parseColumn(getBytes(column));
-          if (famAndQf.length == 1) {
-            get.addFamily(famAndQf[0]);
-          } else {
-            get.addColumn(famAndQf[0], famAndQf[1]);
-          }
-        }
-        get.setTimeRange(0, timestamp);
-        Result result = table.get(get);
-        return ThriftUtilities.rowResultFromHBase(result);
-      } catch (IOException e) {
-        LOG.warn(e.getMessage(), e);
-        throw getIOError(e);
-      } finally{
-        closeTable(table);
-      }
-    }
-
-    @Override
-    public List<TRowResult> getRows(ByteBuffer tableName,
-                                    List<ByteBuffer> rows,
-        Map<ByteBuffer, ByteBuffer> attributes)
-        throws IOError {
-      return getRowsWithColumnsTs(tableName, rows, null,
-                                  HConstants.LATEST_TIMESTAMP,
-                                  attributes);
-    }
-
-    @Override
-    public List<TRowResult> getRowsWithColumns(ByteBuffer tableName,
-                                               List<ByteBuffer> rows,
-        List<ByteBuffer> columns,
-        Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
-      return getRowsWithColumnsTs(tableName, rows, columns,
-                                  HConstants.LATEST_TIMESTAMP,
-                                  attributes);
-    }
-
-    @Override
-    public List<TRowResult> getRowsTs(ByteBuffer tableName,
-                                      List<ByteBuffer> rows,
-        long timestamp,
-        Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
-      return getRowsWithColumnsTs(tableName, rows, null,
-                                  timestamp, attributes);
-    }
-
-    @Override
-    public List<TRowResult> getRowsWithColumnsTs(ByteBuffer tableName,
-                                                 List<ByteBuffer> rows,
-        List<ByteBuffer> columns, long timestamp,
-        Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
-
-      Table table= null;
-      try {
-        List<Get> gets = new ArrayList<>(rows.size());
-        table = getTable(tableName);
-        if (metrics != null) {
-          metrics.incNumRowKeysInBatchGet(rows.size());
-        }
-        for (ByteBuffer row : rows) {
-          Get get = new Get(getBytes(row));
-          addAttributes(get, attributes);
-          if (columns != null) {
-
-            for(ByteBuffer column : columns) {
-              byte [][] famAndQf = CellUtil.parseColumn(getBytes(column));
-              if (famAndQf.length == 1) {
-                get.addFamily(famAndQf[0]);
-              } else {
-                get.addColumn(famAndQf[0], famAndQf[1]);
-              }
-            }
-          }
-          get.setTimeRange(0, timestamp);
-          gets.add(get);
-        }
-        Result[] result = table.get(gets);
-        return ThriftUtilities.rowResultFromHBase(result);
-      } catch (IOException e) {
-        LOG.warn(e.getMessage(), e);
-        throw getIOError(e);
-      } finally{
-        closeTable(table);
-      }
-    }
-
-    @Override
-    public void deleteAll(
-        ByteBuffer tableName, ByteBuffer row, ByteBuffer column,
-        Map<ByteBuffer, ByteBuffer> attributes)
-        throws IOError {
-      deleteAllTs(tableName, row, column, HConstants.LATEST_TIMESTAMP,
-                  attributes);
-    }
-
-    @Override
-    public void deleteAllTs(ByteBuffer tableName,
-                            ByteBuffer row,
-                            ByteBuffer column,
-        long timestamp, Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
-      Table table = null;
-      try {
-        table = getTable(tableName);
-        Delete delete  = new Delete(getBytes(row));
-        addAttributes(delete, attributes);
-        byte [][] famAndQf = CellUtil.parseColumn(getBytes(column));
-        if (famAndQf.length == 1) {
-          delete.addFamily(famAndQf[0], timestamp);
-        } else {
-          delete.addColumns(famAndQf[0], famAndQf[1], timestamp);
-        }
-        table.delete(delete);
-
-      } catch (IOException e) {
-        LOG.warn(e.getMessage(), e);
-        throw getIOError(e);
-      } finally {
-        closeTable(table);
-      }
-    }
-
-    @Override
-    public void deleteAllRow(
-        ByteBuffer tableName, ByteBuffer row,
-        Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
-      deleteAllRowTs(tableName, row, HConstants.LATEST_TIMESTAMP, attributes);
-    }
-
-    @Override
-    public void deleteAllRowTs(
-        ByteBuffer tableName, ByteBuffer row, long timestamp,
-        Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
-      Table table = null;
-      try {
-        table = getTable(tableName);
-        Delete delete  = new Delete(getBytes(row), timestamp);
-        addAttributes(delete, attributes);
-        table.delete(delete);
-      } catch (IOException e) {
-        LOG.warn(e.getMessage(), e);
-        throw getIOError(e);
-      } finally {
-        closeTable(table);
-      }
-    }
-
-    @Override
-    public void createTable(ByteBuffer in_tableName,
-        List<ColumnDescriptor> columnFamilies) throws IOError,
-        IllegalArgument, AlreadyExists {
-      TableName tableName = getTableName(in_tableName);
-      try {
-        if (getAdmin().tableExists(tableName)) {
-          throw new AlreadyExists("table name already in use");
-        }
-        HTableDescriptor desc = new HTableDescriptor(tableName);
-        for (ColumnDescriptor col : columnFamilies) {
-          HColumnDescriptor colDesc = ThriftUtilities.colDescFromThrift(col);
-          desc.addFamily(colDesc);
-        }
-        getAdmin().createTable(desc);
-      } catch (IOException e) {
-        LOG.warn(e.getMessage(), e);
-        throw getIOError(e);
-      } catch (IllegalArgumentException e) {
-        LOG.warn(e.getMessage(), e);
-        throw new IllegalArgument(Throwables.getStackTraceAsString(e));
-      }
-    }
-
-    private static TableName getTableName(ByteBuffer buffer) {
-      return TableName.valueOf(getBytes(buffer));
-    }
-
-    @Override
-    public void deleteTable(ByteBuffer in_tableName) throws IOError {
-      TableName tableName = getTableName(in_tableName);
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("deleteTable: table={}", tableName);
-      }
-      try {
-        if (!getAdmin().tableExists(tableName)) {
-          throw new IOException("table does not exist");
-        }
-        getAdmin().deleteTable(tableName);
-      } catch (IOException e) {
-        LOG.warn(e.getMessage(), e);
-        throw getIOError(e);
-      }
-    }
-
-    @Override
-    public void mutateRow(ByteBuffer tableName, ByteBuffer row,
-        List<Mutation> mutations, Map<ByteBuffer, ByteBuffer> attributes)
-        throws IOError, IllegalArgument {
-      mutateRowTs(tableName, row, mutations, HConstants.LATEST_TIMESTAMP, attributes);
-    }
-
-    @Override
-    public void mutateRowTs(ByteBuffer tableName, ByteBuffer row,
-        List<Mutation> mutations, long timestamp,
-        Map<ByteBuffer, ByteBuffer> attributes)
-        throws IOError, IllegalArgument {
-      Table table = null;
-      try {
-        table = getTable(tableName);
-        Put put = new Put(getBytes(row), timestamp);
-        addAttributes(put, attributes);
-
-        Delete delete = new Delete(getBytes(row));
-        addAttributes(delete, attributes);
-        if (metrics != null) {
-          metrics.incNumRowKeysInBatchMutate(mutations.size());
-        }
-
-        // I apologize for all this mess :)
-        CellBuilder builder = CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY);
-        for (Mutation m : mutations) {
-          byte[][] famAndQf = CellUtil.parseColumn(getBytes(m.column));
-          if (m.isDelete) {
-            if (famAndQf.length == 1) {
-              delete.addFamily(famAndQf[0], timestamp);
-            } else {
-              delete.addColumns(famAndQf[0], famAndQf[1], timestamp);
-            }
-            delete.setDurability(m.writeToWAL ? Durability.SYNC_WAL : Durability.SKIP_WAL);
-          } else {
-            if(famAndQf.length == 1) {
-              LOG.warn("No column qualifier specified. Delete is the only mutation supported "
-                  + "over the whole column family.");
-            } else {
-              put.add(builder.clear()
-                  .setRow(put.getRow())
-                  .setFamily(famAndQf[0])
-                  .setQualifier(famAndQf[1])
-                  .setTimestamp(put.getTimestamp())
-                  .setType(Type.Put)
-                  .setValue(m.value != null ? getBytes(m.value)
-                      : HConstants.EMPTY_BYTE_ARRAY)
-                  .build());
-            }
-            put.setDurability(m.writeToWAL ? Durability.SYNC_WAL : Durability.SKIP_WAL);
-          }
-        }
-        if (!delete.isEmpty()) {
-          table.delete(delete);
-        }
-        if (!put.isEmpty()) {
-          table.put(put);
-        }
-      } catch (IOException e) {
-        LOG.warn(e.getMessage(), e);
-        throw getIOError(e);
-      } catch (IllegalArgumentException e) {
-        LOG.warn(e.getMessage(), e);
-        throw new IllegalArgument(Throwables.getStackTraceAsString(e));
-      } finally{
-        closeTable(table);
-      }
-    }
-
-    @Override
-    public void mutateRows(ByteBuffer tableName, List<BatchMutation> rowBatches,
-        Map<ByteBuffer, ByteBuffer> attributes)
-        throws IOError, IllegalArgument, TException {
-      mutateRowsTs(tableName, rowBatches, HConstants.LATEST_TIMESTAMP, attributes);
-    }
-
-    @Override
-    public void mutateRowsTs(
-        ByteBuffer tableName, List<BatchMutation> rowBatches, long timestamp,
-        Map<ByteBuffer, ByteBuffer> attributes)
-        throws IOError, IllegalArgument, TException {
-      List<Put> puts = new ArrayList<>();
-      List<Delete> deletes = new ArrayList<>();
-      CellBuilder builder = CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY);
-      for (BatchMutation batch : rowBatches) {
-        byte[] row = getBytes(batch.row);
-        List<Mutation> mutations = batch.mutations;
-        Delete delete = new Delete(row);
-        addAttributes(delete, attributes);
-        Put put = new Put(row, timestamp);
-        addAttributes(put, attributes);
-        for (Mutation m : mutations) {
-          byte[][] famAndQf = CellUtil.parseColumn(getBytes(m.column));
-          if (m.isDelete) {
-            // no qualifier, family only.
-            if (famAndQf.length == 1) {
-              delete.addFamily(famAndQf[0], timestamp);
-            } else {
-              delete.addColumns(famAndQf[0], famAndQf[1], timestamp);
-            }
-            delete.setDurability(m.writeToWAL ? Durability.SYNC_WAL
-                : Durability.SKIP_WAL);
-          } else {
-            if (famAndQf.length == 1) {
-              LOG.warn("No column qualifier specified. Delete is the only mutation supported "
-                  + "over the whole column family.");
-            }
-            if (famAndQf.length == 2) {
-              try {
-                put.add(builder.clear()
-                    .setRow(put.getRow())
-                    .setFamily(famAndQf[0])
-                    .setQualifier(famAndQf[1])
-                    .setTimestamp(put.getTimestamp())
-                    .setType(Type.Put)
-                    .setValue(m.value != null ? getBytes(m.value)
-                        : HConstants.EMPTY_BYTE_ARRAY)
-                    .build());
-              } catch (IOException e) {
-                throw new IllegalArgumentException(e);
-              }
-            } else {
-              throw new IllegalArgumentException("Invalid famAndQf provided.");
-            }
-            put.setDurability(m.writeToWAL ? Durability.SYNC_WAL : Durability.SKIP_WAL);
-          }
-        }
-        if (!delete.isEmpty()) {
-          deletes.add(delete);
-        }
-        if (!put.isEmpty()) {
-          puts.add(put);
-        }
-      }
-
-      Table table = null;
-      try {
-        table = getTable(tableName);
-        if (!puts.isEmpty()) {
-          table.put(puts);
-        }
-        if (!deletes.isEmpty()) {
-          table.delete(deletes);
-        }
-      } catch (IOException e) {
-        LOG.warn(e.getMessage(), e);
-        throw getIOError(e);
-      } catch (IllegalArgumentException e) {
-        LOG.warn(e.getMessage(), e);
-        throw new IllegalArgument(Throwables.getStackTraceAsString(e));
-      } finally{
-        closeTable(table);
-      }
-    }
-
-    @Override
-    public long atomicIncrement(
-        ByteBuffer tableName, ByteBuffer row, ByteBuffer column, long amount)
-            throws IOError, IllegalArgument, TException {
-      byte [][] famAndQf = CellUtil.parseColumn(getBytes(column));
-      if(famAndQf.length == 1) {
-        return atomicIncrement(tableName, row, famAndQf[0], HConstants.EMPTY_BYTE_ARRAY, amount);
-      }
-      return atomicIncrement(tableName, row, famAndQf[0], famAndQf[1], amount);
-    }
-
-    protected long atomicIncrement(ByteBuffer tableName, ByteBuffer row,
-        byte [] family, byte [] qualifier, long amount)
-        throws IOError, IllegalArgument, TException {
-      Table table = null;
-      try {
-        table = getTable(tableName);
-        return table.incrementColumnValue(
-            getBytes(row), family, qualifier, amount);
-      } catch (IOException e) {
-        LOG.warn(e.getMessage(), e);
-        throw getIOError(e);
-      } finally {
-        closeTable(table);
-      }
-    }
-
-    @Override
-    public void scannerClose(int id) throws IOError, IllegalArgument {
-      LOG.debug("scannerClose: id={}", id);
-      ResultScannerWrapper resultScannerWrapper = getScanner(id);
-      if (resultScannerWrapper == null) {
-        LOG.warn("scanner ID is invalid");
-        throw new IllegalArgument("scanner ID is invalid");
-      }
-      resultScannerWrapper.getScanner().close();
-      removeScanner(id);
-    }
-
-    @Override
-    public List<TRowResult> scannerGetList(int id,int nbRows)
-        throws IllegalArgument, IOError {
-      LOG.debug("scannerGetList: id={}", id);
-      ResultScannerWrapper resultScannerWrapper = getScanner(id);
-      if (null == resultScannerWrapper) {
-        String message = "scanner ID is invalid";
-        LOG.warn(message);
-        throw new IllegalArgument("scanner ID is invalid");
-      }
-
-      Result [] results;
-      try {
-        results = resultScannerWrapper.getScanner().next(nbRows);
-        if (null == results) {
-          return new ArrayList<>();
-        }
-      } catch (IOException e) {
-        LOG.warn(e.getMessage(), e);
-        throw getIOError(e);
-      }
-      return ThriftUtilities.rowResultFromHBase(results, resultScannerWrapper.isColumnSorted());
-    }
-
-    @Override
-    public List<TRowResult> scannerGet(int id) throws IllegalArgument, IOError {
-      return scannerGetList(id,1);
-    }
-
-    @Override
-    public int scannerOpenWithScan(ByteBuffer tableName, TScan tScan,
-        Map<ByteBuffer, ByteBuffer> attributes)
-        throws IOError {
-
-      Table table = null;
-      try {
-        table = getTable(tableName);
-        Scan scan = new Scan();
-        addAttributes(scan, attributes);
-        if (tScan.isSetStartRow()) {
-          scan.setStartRow(tScan.getStartRow());
-        }
-        if (tScan.isSetStopRow()) {
-          scan.setStopRow(tScan.getStopRow());
-        }
-        if (tScan.isSetTimestamp()) {
-          scan.setTimeRange(0, tScan.getTimestamp());
-        }
-        if (tScan.isSetCaching()) {
-          scan.setCaching(tScan.getCaching());
-        }
-        if (tScan.isSetBatchSize()) {
-          scan.setBatch(tScan.getBatchSize());
-        }
-        if (tScan.isSetColumns() && !tScan.getColumns().isEmpty()) {
-          for(ByteBuffer column : tScan.getColumns()) {
-            byte [][] famQf = CellUtil.parseColumn(getBytes(column));
-            if(famQf.length == 1) {
-              scan.addFamily(famQf[0]);
-            } else {
-              scan.addColumn(famQf[0], famQf[1]);
-            }
-          }
-        }
-        if (tScan.isSetFilterString()) {
-          ParseFilter parseFilter = new ParseFilter();
-          scan.setFilter(
-              parseFilter.parseFilterString(tScan.getFilterString()));
-        }
-        if (tScan.isSetReversed()) {
-          scan.setReversed(tScan.isReversed());
-        }
-        if (tScan.isSetCacheBlocks()) {
-          scan.setCacheBlocks(tScan.isCacheBlocks());
-        }
-        return addScanner(table.getScanner(scan), tScan.sortColumns);
-      } catch (IOException e) {
-        LOG.warn(e.getMessage(), e);
-        throw getIOError(e);
-      } finally{
-        closeTable(table);
-      }
-    }
-
-    @Override
-    public int scannerOpen(ByteBuffer tableName, ByteBuffer startRow,
-        List<ByteBuffer> columns,
-        Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
-
-      Table table = null;
-      try {
-        table = getTable(tableName);
-        Scan scan = new Scan(getBytes(startRow));
-        addAttributes(scan, attributes);
-        if(columns != null && !columns.isEmpty()) {
-          for(ByteBuffer column : columns) {
-            byte [][] famQf = CellUtil.parseColumn(getBytes(column));
-            if(famQf.length == 1) {
-              scan.addFamily(famQf[0]);
-            } else {
-              scan.addColumn(famQf[0], famQf[1]);
-            }
-          }
-        }
-        return addScanner(table.getScanner(scan), false);
-      } catch (IOException e) {
-        LOG.warn(e.getMessage(), e);
-        throw getIOError(e);
-      } finally{
-        closeTable(table);
-      }
-    }
-
-    @Override
-    public int scannerOpenWithStop(ByteBuffer tableName, ByteBuffer startRow,
-        ByteBuffer stopRow, List<ByteBuffer> columns,
-        Map<ByteBuffer, ByteBuffer> attributes)
-        throws IOError, TException {
-
-      Table table = null;
-      try {
-        table = getTable(tableName);
-        Scan scan = new Scan(getBytes(startRow), getBytes(stopRow));
-        addAttributes(scan, attributes);
-        if(columns != null && !columns.isEmpty()) {
-          for(ByteBuffer column : columns) {
-            byte [][] famQf = CellUtil.parseColumn(getBytes(column));
-            if(famQf.length == 1) {
-              scan.addFamily(famQf[0]);
-            } else {
-              scan.addColumn(famQf[0], famQf[1]);
-            }
-          }
-        }
-        return addScanner(table.getScanner(scan), false);
-      } catch (IOException e) {
-        LOG.warn(e.getMessage(), e);
-        throw getIOError(e);
-      } finally{
-        closeTable(table);
-      }
-    }
-
-    @Override
-    public int scannerOpenWithPrefix(ByteBuffer tableName,
-                                     ByteBuffer startAndPrefix,
-                                     List<ByteBuffer> columns,
-        Map<ByteBuffer, ByteBuffer> attributes)
-        throws IOError, TException {
-
-      Table table = null;
-      try {
-        table = getTable(tableName);
-        Scan scan = new Scan(getBytes(startAndPrefix));
-        addAttributes(scan, attributes);
-        Filter f = new WhileMatchFilter(
-            new PrefixFilter(getBytes(startAndPrefix)));
-        scan.setFilter(f);
-        if (columns != null && !columns.isEmpty()) {
-          for(ByteBuffer column : columns) {
-            byte [][] famQf = CellUtil.parseColumn(getBytes(column));
-            if(famQf.length == 1) {
-              scan.addFamily(famQf[0]);
-            } else {
-              scan.addColumn(famQf[0], famQf[1]);
-            }
-          }
-        }
-        return addScanner(table.getScanner(scan), false);
-      } catch (IOException e) {
-        LOG.warn(e.getMessage(), e);
-        throw getIOError(e);
-      } finally{
-        closeTable(table);
-      }
-    }
-
-    @Override
-    public int scannerOpenTs(ByteBuffer tableName, ByteBuffer startRow,
-        List<ByteBuffer> columns, long timestamp,
-        Map<ByteBuffer, ByteBuffer> attributes) throws IOError, TException {
-
-      Table table = null;
-      try {
-        table = getTable(tableName);
-        Scan scan = new Scan(getBytes(startRow));
-        addAttributes(scan, attributes);
-        scan.setTimeRange(0, timestamp);
-        if (columns != null && !columns.isEmpty()) {
-          for (ByteBuffer column : columns) {
-            byte [][] famQf = CellUtil.parseColumn(getBytes(column));
-            if(famQf.length == 1) {
-              scan.addFamily(famQf[0]);
-            } else {
-              scan.addColumn(famQf[0], famQf[1]);
-            }
-          }
-        }
-        return addScanner(table.getScanner(scan), false);
-      } catch (IOException e) {
-        LOG.warn(e.getMessage(), e);
-        throw getIOError(e);
-      } finally{
-        closeTable(table);
-      }
-    }
-
-    @Override
-    public int scannerOpenWithStopTs(ByteBuffer tableName, ByteBuffer startRow,
-        ByteBuffer stopRow, List<ByteBuffer> columns, long timestamp,
-        Map<ByteBuffer, ByteBuffer> attributes)
-        throws IOError, TException {
-
-      Table table = null;
-      try {
-        table = getTable(tableName);
-        Scan scan = new Scan(getBytes(startRow), getBytes(stopRow));
-        addAttributes(scan, attributes);
-        scan.setTimeRange(0, timestamp);
-        if (columns != null && !columns.isEmpty()) {
-          for (ByteBuffer column : columns) {
-            byte [][] famQf = CellUtil.parseColumn(getBytes(column));
-            if(famQf.length == 1) {
-              scan.addFamily(famQf[0]);
-            } else {
-              scan.addColumn(famQf[0], famQf[1]);
-            }
-          }
-        }
-        scan.setTimeRange(0, timestamp);
-        return addScanner(table.getScanner(scan), false);
-      } catch (IOException e) {
-        LOG.warn(e.getMessage(), e);
-        throw getIOError(e);
-      } finally{
-        closeTable(table);
-      }
-    }
-
-    @Override
-    public Map<ByteBuffer, ColumnDescriptor> getColumnDescriptors(
-        ByteBuffer tableName) throws IOError, TException {
-
-      Table table = null;
-      try {
-        TreeMap<ByteBuffer, ColumnDescriptor> columns = new TreeMap<>();
-
-        table = getTable(tableName);
-        HTableDescriptor desc = table.getTableDescriptor();
-
-        for (HColumnDescriptor e : desc.getFamilies()) {
-          ColumnDescriptor col = ThriftUtilities.colDescFromHbase(e);
-          columns.put(col.name, col);
-        }
-        return columns;
-      } catch (IOException e) {
-        LOG.warn(e.getMessage(), e);
-        throw getIOError(e);
-      } finally {
-        closeTable(table);
-      }
-    }
-
-    private void closeTable(Table table) throws IOError {
-      try{
-        if(table != null){
-          table.close();
-        }
-      } catch (IOException e){
-        LOG.error(e.getMessage(), e);
-        throw getIOError(e);
-      }
-    }
-
-    @Override
-    public TRegionInfo getRegionInfo(ByteBuffer searchRow) throws IOError {
-      try {
-        byte[] row = getBytes(searchRow);
-        Result startRowResult = getReverseScanResult(TableName.META_TABLE_NAME.getName(), row,
-          HConstants.CATALOG_FAMILY);
-
-        if (startRowResult == null) {
-          throw new IOException("Cannot find row in "+ TableName.META_TABLE_NAME+", row="
-                                + Bytes.toStringBinary(row));
-        }
-
-        // find region start and end keys
-        RegionInfo regionInfo = MetaTableAccessor.getRegionInfo(startRowResult);
-        if (regionInfo == null) {
-          throw new IOException("RegionInfo REGIONINFO was null or " +
-                                " empty in Meta for row="
-                                + Bytes.toStringBinary(row));
-        }
-        TRegionInfo region = new TRegionInfo();
-        region.setStartKey(regionInfo.getStartKey());
-        region.setEndKey(regionInfo.getEndKey());
-        region.id = regionInfo.getRegionId();
-        region.setName(regionInfo.getRegionName());
-        region.version = HREGION_VERSION; // version not used anymore, PB encoding used.
-
-        // find region assignment to server
-        ServerName serverName = MetaTableAccessor.getServerName(startRowResult, 0);
-        if (serverName != null) {
-          region.setServerName(Bytes.toBytes(serverName.getHostname()));
-          region.port = serverName.getPort();
-        }
-        return region;
-      } catch (IOException e) {
-        LOG.warn(e.getMessage(), e);
-        throw getIOError(e);
-      }
-    }
-
-    private Result getReverseScanResult(byte[] tableName, byte[] row, byte[] family)
-        throws IOException {
-      Scan scan = new Scan(row);
-      scan.setReversed(true);
-      scan.addFamily(family);
-      scan.setStartRow(row);
-      try (Table table = getTable(tableName);
-           ResultScanner scanner = table.getScanner(scan)) {
-        return scanner.next();
-      }
-    }
-
-    private void initMetrics(ThriftMetrics metrics) {
-      this.metrics = metrics;
-    }
-
-    @Override
-    public void increment(TIncrement tincrement) throws IOError, TException {
-
-      if (tincrement.getRow().length == 0 || tincrement.getTable().length == 0) {
-        throw new TException("Must supply a table and a row key; can't increment");
-      }
-
-      if (conf.getBoolean(COALESCE_INC_KEY, false)) {
-        this.coalescer.queueIncrement(tincrement);
-        return;
-      }
-
-      Table table = null;
-      try {
-        table = getTable(tincrement.getTable());
-        Increment inc = ThriftUtilities.incrementFromThrift(tincrement);
-        table.increment(inc);
-      } catch (IOException e) {
-        LOG.warn(e.getMessage(), e);
-        throw getIOError(e);
-      } finally{
-        closeTable(table);
-      }
-    }
-
-    @Override
-    public void incrementRows(List<TIncrement> tincrements) throws IOError, TException {
-      if (conf.getBoolean(COALESCE_INC_KEY, false)) {
-        this.coalescer.queueIncrements(tincrements);
-        return;
-      }
-      for (TIncrement tinc : tincrements) {
-        increment(tinc);
-      }
-    }
-
-    @Override
-    public List<TCell> append(TAppend tappend) throws IOError, TException {
-      if (tappend.getRow().length == 0 || tappend.getTable().length == 0) {
-        throw new TException("Must supply a table and a row key; can't append");
-      }
-
-      Table table = null;
-      try {
-        table = getTable(tappend.getTable());
-        Append append = ThriftUtilities.appendFromThrift(tappend);
-        Result result = table.append(append);
-        return ThriftUtilities.cellFromHBase(result.rawCells());
-      } catch (IOException e) {
-        LOG.warn(e.getMessage(), e);
-        throw getIOError(e);
-      } finally{
-        closeTable(table);
-      }
-    }
-
-    @Override
-    public boolean checkAndPut(ByteBuffer tableName, ByteBuffer row, ByteBuffer column,
-        ByteBuffer value, Mutation mput, Map<ByteBuffer, ByteBuffer> attributes) throws IOError,
-        IllegalArgument, TException {
-      Put put;
-      try {
-        put = new Put(getBytes(row), HConstants.LATEST_TIMESTAMP);
-        addAttributes(put, attributes);
-
-        byte[][] famAndQf = CellUtil.parseColumn(getBytes(mput.column));
-        put.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY)
-            .setRow(put.getRow())
-            .setFamily(famAndQf[0])
-            .setQualifier(famAndQf[1])
-            .setTimestamp(put.getTimestamp())
-            .setType(Type.Put)
-            .setValue(mput.value != null ? getBytes(mput.value)
-                : HConstants.EMPTY_BYTE_ARRAY)
-            .build());
-        put.setDurability(mput.writeToWAL ? Durability.SYNC_WAL : Durability.SKIP_WAL);
-      } catch (IOException | IllegalArgumentException e) {
-        LOG.warn(e.getMessage(), e);
-        throw new IllegalArgument(Throwables.getStackTraceAsString(e));
-      }
-
-      Table table = null;
-      try {
-        table = getTable(tableName);
-        byte[][] famAndQf = CellUtil.parseColumn(getBytes(column));
-        Table.CheckAndMutateBuilder mutateBuilder =
-            table.checkAndMutate(getBytes(row), famAndQf[0]).qualifier(famAndQf[1]);
-        if (value != null) {
-          return mutateBuilder.ifEquals(getBytes(value)).thenPut(put);
-        } else {
-          return mutateBuilder.ifNotExists().thenPut(put);
-        }
-      } catch (IOException e) {
-        LOG.warn(e.getMessage(), e);
-        throw getIOError(e);
-      } catch (IllegalArgumentException e) {
-        LOG.warn(e.getMessage(), e);
-        throw new IllegalArgument(Throwables.getStackTraceAsString(e));
-      } finally {
-        closeTable(table);
-      }
-    }
-  }
-
-  private static IOError getIOError(Throwable throwable) {
-    IOError error = new IOErrorWithCause(throwable);
-    error.setMessage(Throwables.getStackTraceAsString(throwable));
-    return error;
-  }
-
-  /**
-   * Adds all the attributes into the Operation object
-   */
-  private static void addAttributes(OperationWithAttributes op,
-    Map<ByteBuffer, ByteBuffer> attributes) {
-    if (attributes == null || attributes.isEmpty()) {
-      return;
-    }
-    for (Map.Entry<ByteBuffer, ByteBuffer> entry : attributes.entrySet()) {
-      String name = Bytes.toStringBinary(getBytes(entry.getKey()));
-      byte[] value =  getBytes(entry.getValue());
-      op.setAttribute(name, value);
-    }
-  }
-
-  public static void registerFilters(Configuration conf) {
-    String[] filters = conf.getStrings("hbase.thrift.filters");
-    Splitter splitter = Splitter.on(':');
-    if(filters != null) {
-      for(String filterClass: filters) {
-        List<String> filterPart = splitter.splitToList(filterClass);
-        if(filterPart.size() != 2) {
-          LOG.warn("Invalid filter specification " + filterClass + " - skipping");
-        } else {
-          ParseFilter.registerFilter(filterPart.get(0), filterPart.get(1));
-        }
-      }
-    }
-  }
-
-  public static class IOErrorWithCause extends IOError {
-    private final Throwable cause;
-    public IOErrorWithCause(Throwable cause) {
-      this.cause = cause;
-    }
-
-    @Override
-    public synchronized Throwable getCause() {
-      return cause;
-    }
-
-    @Override
-    public boolean equals(Object other) {
-      if (super.equals(other) &&
-          other instanceof IOErrorWithCause) {
-        Throwable otherCause = ((IOErrorWithCause) other).getCause();
-        if (this.getCause() != null) {
-          return otherCause != null && this.getCause().equals(otherCause);
-        } else {
-          return otherCause == null;
-        }
-      }
-      return false;
-    }
-
-    @Override
-    public int hashCode() {
-      int result = super.hashCode();
-      result = 31 * result + (cause != null ? cause.hashCode() : 0);
-      return result;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b6b4af/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftHBaseServiceHandler.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftHBaseServiceHandler.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftHBaseServiceHandler.java
index 2bfeefe..a9ec646 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftHBaseServiceHandler.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftHBaseServiceHandler.java
@@ -18,6 +18,8 @@
  */
 package org.apache.hadoop.hbase.thrift2;
 
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_READONLY_ENABLED;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_READONLY_ENABLED_DEFAULT;
 import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.appendFromThrift;
 import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.columnFamilyDescriptorFromThrift;
 import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.compareOpFromThrift;
@@ -44,10 +46,6 @@ import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.tableNamesFromHBas
 import static org.apache.thrift.TBaseHelper.byteBufferToByteArray;
 
 import java.io.IOException;
-import java.lang.reflect.InvocationHandler;
-import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Method;
-import java.lang.reflect.Proxy;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -68,7 +66,7 @@ import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.security.UserProvider;
-import org.apache.hadoop.hbase.thrift.ThriftMetrics;
+import org.apache.hadoop.hbase.thrift.HBaseServiceHandler;
 import org.apache.hadoop.hbase.thrift2.generated.TAppend;
 import org.apache.hadoop.hbase.thrift2.generated.TColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.thrift2.generated.TCompareOp;
@@ -87,7 +85,6 @@ import org.apache.hadoop.hbase.thrift2.generated.TScan;
 import org.apache.hadoop.hbase.thrift2.generated.TTableDescriptor;
 import org.apache.hadoop.hbase.thrift2.generated.TTableName;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.ConnectionCache;
 import org.apache.thrift.TException;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
@@ -99,7 +96,7 @@ import org.slf4j.LoggerFactory;
  */
 @InterfaceAudience.Private
 @SuppressWarnings("deprecation")
-public class ThriftHBaseServiceHandler implements THBaseService.Iface {
+public class ThriftHBaseServiceHandler extends HBaseServiceHandler implements THBaseService.Iface {
 
   // TODO: Size of pool configuraple
   private static final Logger LOG = LoggerFactory.getLogger(ThriftHBaseServiceHandler.class);
@@ -109,50 +106,10 @@ public class ThriftHBaseServiceHandler implements THBaseService.Iface {
   private final AtomicInteger nextScannerId = new AtomicInteger(0);
   private final Map<Integer, ResultScanner> scannerMap = new ConcurrentHashMap<>();
 
-  private final ConnectionCache connectionCache;
-
-  static final String CLEANUP_INTERVAL = "hbase.thrift.connection.cleanup-interval";
-  static final String MAX_IDLETIME = "hbase.thrift.connection.max-idletime";
-
   private static final IOException ioe
       = new DoNotRetryIOException("Thrift Server is in Read-only mode.");
   private boolean isReadOnly;
 
-  public static THBaseService.Iface newInstance(
-      THBaseService.Iface handler, ThriftMetrics metrics) {
-    return (THBaseService.Iface) Proxy.newProxyInstance(handler.getClass().getClassLoader(),
-      new Class[] { THBaseService.Iface.class }, new THBaseServiceMetricsProxy(handler, metrics));
-  }
-
-  private static final class THBaseServiceMetricsProxy implements InvocationHandler {
-    private final THBaseService.Iface handler;
-    private final ThriftMetrics metrics;
-
-    private THBaseServiceMetricsProxy(THBaseService.Iface handler, ThriftMetrics metrics) {
-      this.handler = handler;
-      this.metrics = metrics;
-    }
-
-    @Override
-    public Object invoke(Object proxy, Method m, Object[] args) throws Throwable {
-      Object result;
-      long start = now();
-      try {
-        result = m.invoke(handler, args);
-      } catch (InvocationTargetException e) {
-        metrics.exception(e.getCause());
-        throw e.getTargetException();
-      } catch (Exception e) {
-        metrics.exception(e);
-        throw new RuntimeException("unexpected invocation exception: " + e.getMessage());
-      } finally {
-        long processTime = now() - start;
-        metrics.incMethodTime(m.getName(), processTime);
-      }
-      return result;
-    }
-  }
-
   private static class TIOErrorWithCause extends TIOError {
     private Throwable cause;
 
@@ -188,20 +145,14 @@ public class ThriftHBaseServiceHandler implements THBaseService.Iface {
     }
   }
 
-  private static long now() {
-    return System.nanoTime();
-  }
-
   ThriftHBaseServiceHandler(final Configuration conf,
       final UserProvider userProvider) throws IOException {
-    int cleanInterval = conf.getInt(CLEANUP_INTERVAL, 10 * 1000);
-    int maxIdleTime = conf.getInt(MAX_IDLETIME, 10 * 60 * 1000);
-    connectionCache = new ConnectionCache(
-      conf, userProvider, cleanInterval, maxIdleTime);
-    isReadOnly = conf.getBoolean("hbase.thrift.readonly", false);
+    super(conf, userProvider);
+    isReadOnly = conf.getBoolean(THRIFT_READONLY_ENABLED, THRIFT_READONLY_ENABLED_DEFAULT);
   }
 
-  private Table getTable(ByteBuffer tableName) {
+  @Override
+  protected Table getTable(ByteBuffer tableName) {
     try {
       return connectionCache.getTable(Bytes.toString(byteBufferToByteArray(tableName)));
     } catch (IOException ie) {
@@ -251,10 +202,6 @@ public class ThriftHBaseServiceHandler implements THBaseService.Iface {
     return scannerMap.get(id);
   }
 
-  void setEffectiveUser(String effectiveUser) {
-    connectionCache.setEffectiveUser(effectiveUser);
-  }
-
   /**
    * Removes the scanner associated with the specified ID from the internal HashMap.
    * @param id of the Scanner to remove


[04/15] hbase git commit: HBASE-21652 Refactor ThriftServer making thrift2 server inherited from thrift1 server

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b6b4af/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServer.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServer.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServer.java
index fc00327..6d11ac6 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServer.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServer.java
@@ -18,16 +18,132 @@
 
 package org.apache.hadoop.hbase.thrift;
 
+import static org.apache.hadoop.hbase.thrift.Constants.BACKLOG_CONF_DEAFULT;
+import static org.apache.hadoop.hbase.thrift.Constants.BACKLOG_CONF_KEY;
+import static org.apache.hadoop.hbase.thrift.Constants.BIND_CONF_KEY;
+import static org.apache.hadoop.hbase.thrift.Constants.BIND_OPTION;
+import static org.apache.hadoop.hbase.thrift.Constants.COMPACT_CONF_DEFAULT;
+import static org.apache.hadoop.hbase.thrift.Constants.COMPACT_CONF_KEY;
+import static org.apache.hadoop.hbase.thrift.Constants.COMPACT_OPTION;
+import static org.apache.hadoop.hbase.thrift.Constants.DEFAULT_BIND_ADDR;
+import static org.apache.hadoop.hbase.thrift.Constants.DEFAULT_HTTP_MAX_HEADER_SIZE;
+import static org.apache.hadoop.hbase.thrift.Constants.DEFAULT_LISTEN_PORT;
+import static org.apache.hadoop.hbase.thrift.Constants.FRAMED_CONF_DEFAULT;
+import static org.apache.hadoop.hbase.thrift.Constants.FRAMED_CONF_KEY;
+import static org.apache.hadoop.hbase.thrift.Constants.FRAMED_OPTION;
+import static org.apache.hadoop.hbase.thrift.Constants.HTTP_MAX_THREADS_KEY;
+import static org.apache.hadoop.hbase.thrift.Constants.HTTP_MAX_THREADS_KEY_DEFAULT;
+import static org.apache.hadoop.hbase.thrift.Constants.HTTP_MIN_THREADS_KEY;
+import static org.apache.hadoop.hbase.thrift.Constants.HTTP_MIN_THREADS_KEY_DEFAULT;
+import static org.apache.hadoop.hbase.thrift.Constants.INFOPORT_OPTION;
+import static org.apache.hadoop.hbase.thrift.Constants.KEEP_ALIVE_SEC_OPTION;
+import static org.apache.hadoop.hbase.thrift.Constants.MAX_FRAME_SIZE_CONF_DEFAULT;
+import static org.apache.hadoop.hbase.thrift.Constants.MAX_FRAME_SIZE_CONF_KEY;
+import static org.apache.hadoop.hbase.thrift.Constants.MAX_QUEUE_SIZE_OPTION;
+import static org.apache.hadoop.hbase.thrift.Constants.MAX_WORKERS_OPTION;
+import static org.apache.hadoop.hbase.thrift.Constants.MIN_WORKERS_OPTION;
+import static org.apache.hadoop.hbase.thrift.Constants.PORT_CONF_KEY;
+import static org.apache.hadoop.hbase.thrift.Constants.PORT_OPTION;
+import static org.apache.hadoop.hbase.thrift.Constants.READ_TIMEOUT_OPTION;
+import static org.apache.hadoop.hbase.thrift.Constants.SELECTOR_NUM_OPTION;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_DNS_INTERFACE_KEY;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_DNS_NAMESERVER_KEY;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_FILTERS;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_HTTP_ALLOW_OPTIONS_METHOD;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_HTTP_ALLOW_OPTIONS_METHOD_DEFAULT;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_INFO_SERVER_BINDING_ADDRESS;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_INFO_SERVER_BINDING_ADDRESS_DEFAULT;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_INFO_SERVER_PORT;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_INFO_SERVER_PORT_DEFAULT;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_KERBEROS_PRINCIPAL_KEY;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_KEYTAB_FILE_KEY;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_QOP_KEY;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_SELECTOR_NUM;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_SERVER_SOCKET_READ_TIMEOUT_DEFAULT;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_SERVER_SOCKET_READ_TIMEOUT_KEY;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_SSL_ENABLED_KEY;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_SSL_EXCLUDE_CIPHER_SUITES_KEY;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_SSL_EXCLUDE_PROTOCOLS_KEY;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_SSL_INCLUDE_CIPHER_SUITES_KEY;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_SSL_INCLUDE_PROTOCOLS_KEY;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_SSL_KEYSTORE_KEYPASSWORD_KEY;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_SSL_KEYSTORE_PASSWORD_KEY;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_SSL_KEYSTORE_STORE_KEY;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_SUPPORT_PROXYUSER_KEY;
+import static org.apache.hadoop.hbase.thrift.Constants.USE_HTTP_CONF_KEY;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.UnknownHostException;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.sasl.AuthorizeCallback;
+import javax.security.sasl.SaslServer;
+
+import org.apache.commons.lang3.ArrayUtils;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.filter.ParseFilter;
+import org.apache.hadoop.hbase.http.HttpServerUtil;
 import org.apache.hadoop.hbase.http.InfoServer;
-import org.apache.hadoop.hbase.thrift.ThriftServerRunner.ImplType;
+import org.apache.hadoop.hbase.security.SaslUtil;
+import org.apache.hadoop.hbase.security.SecurityUtil;
+import org.apache.hadoop.hbase.security.UserProvider;
+import org.apache.hadoop.hbase.thrift.generated.Hbase;
+import org.apache.hadoop.hbase.util.DNS;
+import org.apache.hadoop.hbase.util.JvmPauseMonitor;
+import org.apache.hadoop.hbase.util.Strings;
 import org.apache.hadoop.hbase.util.VersionInfo;
+import org.apache.hadoop.security.SaslRpcServer;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authorize.ProxyUsers;
 import org.apache.hadoop.util.Shell.ExitCodeException;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.thrift.TProcessor;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.protocol.TProtocolFactory;
+import org.apache.thrift.server.THsHaServer;
+import org.apache.thrift.server.TNonblockingServer;
+import org.apache.thrift.server.TServer;
+import org.apache.thrift.server.TServlet;
+import org.apache.thrift.server.TThreadedSelectorServer;
+import org.apache.thrift.transport.TFramedTransport;
+import org.apache.thrift.transport.TNonblockingServerSocket;
+import org.apache.thrift.transport.TNonblockingServerTransport;
+import org.apache.thrift.transport.TSaslServerTransport;
+import org.apache.thrift.transport.TServerSocket;
+import org.apache.thrift.transport.TServerTransport;
+import org.apache.thrift.transport.TTransportFactory;
 import org.apache.yetus.audience.InterfaceAudience;
+import org.eclipse.jetty.http.HttpVersion;
+import org.eclipse.jetty.server.HttpConfiguration;
+import org.eclipse.jetty.server.HttpConnectionFactory;
+import org.eclipse.jetty.server.SecureRequestCustomizer;
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.server.ServerConnector;
+import org.eclipse.jetty.server.SslConnectionFactory;
+import org.eclipse.jetty.servlet.ServletContextHandler;
+import org.eclipse.jetty.servlet.ServletHolder;
+import org.eclipse.jetty.util.ssl.SslContextFactory;
+import org.eclipse.jetty.util.thread.QueuedThreadPool;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.base.Joiner;
+import org.apache.hbase.thirdparty.com.google.common.base.Splitter;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
 import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
 import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLineParser;
 import org.apache.hbase.thirdparty.org.apache.commons.cli.DefaultParser;
@@ -40,29 +156,36 @@ import org.apache.hbase.thirdparty.org.apache.commons.cli.Options;
  * independent process.
  */
 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
-public class ThriftServer {
+public class ThriftServer  extends Configured implements Tool {
 
   private static final Logger LOG = LoggerFactory.getLogger(ThriftServer.class);
 
-  private static final String MIN_WORKERS_OPTION = "minWorkers";
-  private static final String MAX_WORKERS_OPTION = "workers";
-  private static final String MAX_QUEUE_SIZE_OPTION = "queue";
-  private static final String KEEP_ALIVE_SEC_OPTION = "keepAliveSec";
-  static final String BIND_OPTION = "bind";
-  static final String COMPACT_OPTION = "compact";
-  static final String FRAMED_OPTION = "framed";
-  static final String PORT_OPTION = "port";
-  static final String INFOPORT_OPTION = "infoport";
 
-  private static final String DEFAULT_BIND_ADDR = "0.0.0.0";
-  private static final int DEFAULT_LISTEN_PORT = 9090;
 
-  private Configuration conf;
-  ThriftServerRunner serverRunner;
+  protected Configuration conf;
+
+  protected InfoServer infoServer;
+
+  protected TProcessor processor;
+
+  protected ThriftMetrics metrics;
+  protected HBaseServiceHandler hbaseServiceHandler;
+  protected UserGroupInformation serviceUGI;
+  protected boolean httpEnabled;
+
+  protected SaslUtil.QualityOfProtection qop;
+  protected String host;
+  protected int listenPort;
+
+
+  protected boolean securityEnabled;
+  protected boolean doAsEnabled;
+
+  protected JvmPauseMonitor pauseMonitor;
 
-  private InfoServer infoServer;
+  protected volatile TServer tserver;
+  protected volatile Server httpServer;
 
-  private static final String READ_TIMEOUT_OPTION = "readTimeout";
 
   //
   // Main program and support routines
@@ -72,7 +195,89 @@ public class ThriftServer {
     this.conf = HBaseConfiguration.create(conf);
   }
 
-  private static void printUsageAndExit(Options options, int exitCode)
+  protected void setupParamters() throws IOException {
+    // login the server principal (if using secure Hadoop)
+    UserProvider userProvider = UserProvider.instantiate(conf);
+    securityEnabled = userProvider.isHadoopSecurityEnabled()
+        && userProvider.isHBaseSecurityEnabled();
+    if (securityEnabled) {
+      host = Strings.domainNamePointerToHostName(DNS.getDefaultHost(
+          conf.get(THRIFT_DNS_INTERFACE_KEY, "default"),
+          conf.get(THRIFT_DNS_NAMESERVER_KEY, "default")));
+      userProvider.login(THRIFT_KEYTAB_FILE_KEY, THRIFT_KERBEROS_PRINCIPAL_KEY, host);
+    }
+    this.serviceUGI = userProvider.getCurrent().getUGI();
+
+    this.listenPort = conf.getInt(PORT_CONF_KEY, DEFAULT_LISTEN_PORT);
+    this.metrics = new ThriftMetrics(conf, ThriftMetrics.ThriftServerType.ONE);
+    this.pauseMonitor = new JvmPauseMonitor(conf, this.metrics.getSource());
+    this.hbaseServiceHandler = createHandler(conf, userProvider);
+    this.hbaseServiceHandler.initMetrics(metrics);
+    this.processor = createProcessor();
+
+    httpEnabled = conf.getBoolean(USE_HTTP_CONF_KEY, false);
+    doAsEnabled = conf.getBoolean(THRIFT_SUPPORT_PROXYUSER_KEY, false);
+    if (doAsEnabled && !httpEnabled) {
+      LOG.warn("Fail to enable the doAs feature. " + USE_HTTP_CONF_KEY + " is not configured");
+    }
+
+    String strQop = conf.get(THRIFT_QOP_KEY);
+    if (strQop != null) {
+      this.qop = SaslUtil.getQop(strQop);
+    }
+    if (qop != null) {
+      if (qop != SaslUtil.QualityOfProtection.AUTHENTICATION &&
+          qop != SaslUtil.QualityOfProtection.INTEGRITY &&
+          qop != SaslUtil.QualityOfProtection.PRIVACY) {
+        throw new IOException(String.format("Invalid %s: It must be one of %s, %s, or %s.",
+            THRIFT_QOP_KEY,
+            SaslUtil.QualityOfProtection.AUTHENTICATION.name(),
+            SaslUtil.QualityOfProtection.INTEGRITY.name(),
+            SaslUtil.QualityOfProtection.PRIVACY.name()));
+      }
+      checkHttpSecurity(qop, conf);
+      if (!securityEnabled) {
+        throw new IOException("Thrift server must run in secure mode to support authentication");
+      }
+    }
+    registerFilters(conf);
+    pauseMonitor.start();
+  }
+
+  protected void startInfoServer() throws IOException {
+    // Put up info server.
+    int port = conf.getInt(THRIFT_INFO_SERVER_PORT , THRIFT_INFO_SERVER_PORT_DEFAULT);
+
+    if (port >= 0) {
+      conf.setLong("startcode", System.currentTimeMillis());
+      String a = conf
+          .get(THRIFT_INFO_SERVER_BINDING_ADDRESS, THRIFT_INFO_SERVER_BINDING_ADDRESS_DEFAULT);
+      infoServer = new InfoServer("thrift", a, port, false, conf);
+      infoServer.setAttribute("hbase.conf", conf);
+      infoServer.start();
+    }
+  }
+
+  protected void checkHttpSecurity(SaslUtil.QualityOfProtection qop, Configuration conf) {
+    if (qop == SaslUtil.QualityOfProtection.PRIVACY &&
+        conf.getBoolean(USE_HTTP_CONF_KEY, false) &&
+        !conf.getBoolean(THRIFT_SSL_ENABLED_KEY, false)) {
+      throw new IllegalArgumentException("Thrift HTTP Server's QoP is privacy, but " +
+          THRIFT_SSL_ENABLED_KEY + " is false");
+    }
+  }
+
+  protected HBaseServiceHandler createHandler(Configuration conf, UserProvider userProvider)
+      throws IOException {
+    return new ThriftHBaseServiceHandler(conf, userProvider);
+  }
+
+  protected TProcessor createProcessor() {
+    return new Hbase.Processor<>(
+        HbaseHandlerMetricsProxy.newInstance((Hbase.Iface) hbaseServiceHandler, metrics, conf));
+  }
+
+  protected void printUsageAndExit(Options options, int exitCode)
       throws ExitCodeException {
     HelpFormatter formatter = new HelpFormatter();
     formatter.printHelp("Thrift", null, options,
@@ -85,32 +290,333 @@ public class ThriftServer {
   }
 
   /**
-   * Start up or shuts down the Thrift server, depending on the arguments.
-   * @param args the arguments to pass in when starting the Thrift server
+   * Setup a HTTP Server using Jetty to serve calls from THttpClient
+   *
+   * @throws IOException IOException
    */
-  void doMain(final String[] args) throws Exception {
-    processOptions(args);
-    serverRunner = new ThriftServerRunner(conf);
+  protected void setupHTTPServer() throws IOException {
+    TProtocolFactory protocolFactory = new TBinaryProtocol.Factory();
+    TServlet thriftHttpServlet = new ThriftHttpServlet(processor, protocolFactory, serviceUGI,
+        conf, hbaseServiceHandler, securityEnabled, doAsEnabled);
+
+    // Set the default max thread number to 100 to limit
+    // the number of concurrent requests so that Thrfit HTTP server doesn't OOM easily.
+    // Jetty set the default max thread number to 250, if we don't set it.
+    //
+    // Our default min thread number 2 is the same as that used by Jetty.
+    int minThreads = conf.getInt(HTTP_MIN_THREADS_KEY,
+        conf.getInt(TBoundedThreadPoolServer.MIN_WORKER_THREADS_CONF_KEY,
+            HTTP_MIN_THREADS_KEY_DEFAULT));
+    int maxThreads = conf.getInt(HTTP_MAX_THREADS_KEY,
+        conf.getInt(TBoundedThreadPoolServer.MAX_WORKER_THREADS_CONF_KEY,
+            HTTP_MAX_THREADS_KEY_DEFAULT));
+    QueuedThreadPool threadPool = new QueuedThreadPool(maxThreads);
+    threadPool.setMinThreads(minThreads);
+    httpServer = new Server(threadPool);
+
+    // Context handler
+    ServletContextHandler ctxHandler = new ServletContextHandler(httpServer, "/",
+        ServletContextHandler.SESSIONS);
+    ctxHandler.addServlet(new ServletHolder(thriftHttpServlet), "/*");
+    HttpServerUtil.constrainHttpMethods(ctxHandler,
+        conf.getBoolean(THRIFT_HTTP_ALLOW_OPTIONS_METHOD,
+            THRIFT_HTTP_ALLOW_OPTIONS_METHOD_DEFAULT));
+
+    // set up Jetty and run the embedded server
+    HttpConfiguration httpConfig = new HttpConfiguration();
+    httpConfig.setSecureScheme("https");
+    httpConfig.setSecurePort(listenPort);
+    httpConfig.setHeaderCacheSize(DEFAULT_HTTP_MAX_HEADER_SIZE);
+    httpConfig.setRequestHeaderSize(DEFAULT_HTTP_MAX_HEADER_SIZE);
+    httpConfig.setResponseHeaderSize(DEFAULT_HTTP_MAX_HEADER_SIZE);
+    httpConfig.setSendServerVersion(false);
+    httpConfig.setSendDateHeader(false);
+
+    ServerConnector serverConnector;
+    if(conf.getBoolean(THRIFT_SSL_ENABLED_KEY, false)) {
+      HttpConfiguration httpsConfig = new HttpConfiguration(httpConfig);
+      httpsConfig.addCustomizer(new SecureRequestCustomizer());
+
+      SslContextFactory sslCtxFactory = new SslContextFactory();
+      String keystore = conf.get(THRIFT_SSL_KEYSTORE_STORE_KEY);
+      String password = HBaseConfiguration.getPassword(conf,
+          THRIFT_SSL_KEYSTORE_PASSWORD_KEY, null);
+      String keyPassword = HBaseConfiguration.getPassword(conf,
+          THRIFT_SSL_KEYSTORE_KEYPASSWORD_KEY, password);
+      sslCtxFactory.setKeyStorePath(keystore);
+      sslCtxFactory.setKeyStorePassword(password);
+      sslCtxFactory.setKeyManagerPassword(keyPassword);
+
+      String[] excludeCiphers = conf.getStrings(
+          THRIFT_SSL_EXCLUDE_CIPHER_SUITES_KEY, ArrayUtils.EMPTY_STRING_ARRAY);
+      if (excludeCiphers.length != 0) {
+        sslCtxFactory.setExcludeCipherSuites(excludeCiphers);
+      }
+      String[] includeCiphers = conf.getStrings(
+          THRIFT_SSL_INCLUDE_CIPHER_SUITES_KEY, ArrayUtils.EMPTY_STRING_ARRAY);
+      if (includeCiphers.length != 0) {
+        sslCtxFactory.setIncludeCipherSuites(includeCiphers);
+      }
 
-    // Put up info server.
-    int port = conf.getInt("hbase.thrift.info.port", 9095);
+      // Disable SSLv3 by default due to "Poodle" Vulnerability - CVE-2014-3566
+      String[] excludeProtocols = conf.getStrings(
+          THRIFT_SSL_EXCLUDE_PROTOCOLS_KEY, "SSLv3");
+      if (excludeProtocols.length != 0) {
+        sslCtxFactory.setExcludeProtocols(excludeProtocols);
+      }
+      String[] includeProtocols = conf.getStrings(
+          THRIFT_SSL_INCLUDE_PROTOCOLS_KEY, ArrayUtils.EMPTY_STRING_ARRAY);
+      if (includeProtocols.length != 0) {
+        sslCtxFactory.setIncludeProtocols(includeProtocols);
+      }
 
-    if (port >= 0) {
-      conf.setLong("startcode", System.currentTimeMillis());
-      String a = conf.get("hbase.thrift.info.bindAddress", "0.0.0.0");
-      infoServer = new InfoServer("thrift", a, port, false, conf);
-      infoServer.setAttribute("hbase.conf", conf);
-      infoServer.start();
+      serverConnector = new ServerConnector(httpServer,
+          new SslConnectionFactory(sslCtxFactory, HttpVersion.HTTP_1_1.toString()),
+          new HttpConnectionFactory(httpsConfig));
+    } else {
+      serverConnector = new ServerConnector(httpServer, new HttpConnectionFactory(httpConfig));
     }
+    serverConnector.setPort(listenPort);
+    serverConnector.setHost(getBindAddress(conf).getHostAddress());
+    httpServer.addConnector(serverConnector);
+    httpServer.setStopAtShutdown(true);
 
-    serverRunner.run();
+    if (doAsEnabled) {
+      ProxyUsers.refreshSuperUserGroupsConfiguration(conf);
+    }
+    LOG.info("Starting Thrift HTTP Server on {}", Integer.toString(listenPort));
   }
 
   /**
-   * Parse the command line options to set parameters the conf.
+   * Setting up the thrift TServer
    */
-  private void processOptions(final String[] args) throws Exception {
-    Options options = new Options();
+  protected void setupServer() throws Exception {
+    // Construct correct ProtocolFactory
+    TProtocolFactory protocolFactory = getProtocolFactory();
+
+    ImplType implType = ImplType.getServerImpl(conf);
+    TProcessor processorToUse = processor;
+
+    // Construct correct TransportFactory
+    TTransportFactory transportFactory;
+    if (conf.getBoolean(FRAMED_CONF_KEY, FRAMED_CONF_DEFAULT) || implType.isAlwaysFramed) {
+      if (qop != null) {
+        throw new RuntimeException("Thrift server authentication"
+            + " doesn't work with framed transport yet");
+      }
+      transportFactory = new TFramedTransport.Factory(
+          conf.getInt(MAX_FRAME_SIZE_CONF_KEY, MAX_FRAME_SIZE_CONF_DEFAULT) * 1024 * 1024);
+      LOG.debug("Using framed transport");
+    } else if (qop == null) {
+      transportFactory = new TTransportFactory();
+    } else {
+      // Extract the name from the principal
+      String thriftKerberosPrincipal = conf.get(THRIFT_KERBEROS_PRINCIPAL_KEY);
+      if (thriftKerberosPrincipal == null) {
+        throw new IllegalArgumentException(THRIFT_KERBEROS_PRINCIPAL_KEY + " cannot be null");
+      }
+      String name = SecurityUtil.getUserFromPrincipal(thriftKerberosPrincipal);
+      Map<String, String> saslProperties = SaslUtil.initSaslProperties(qop.name());
+      TSaslServerTransport.Factory saslFactory = new TSaslServerTransport.Factory();
+      saslFactory.addServerDefinition("GSSAPI", name, host, saslProperties,
+          new SaslRpcServer.SaslGssCallbackHandler() {
+            @Override
+            public void handle(Callback[] callbacks)
+                throws UnsupportedCallbackException {
+              AuthorizeCallback ac = null;
+              for (Callback callback : callbacks) {
+                if (callback instanceof AuthorizeCallback) {
+                  ac = (AuthorizeCallback) callback;
+                } else {
+                  throw new UnsupportedCallbackException(callback,
+                      "Unrecognized SASL GSSAPI Callback");
+                }
+              }
+              if (ac != null) {
+                String authid = ac.getAuthenticationID();
+                String authzid = ac.getAuthorizationID();
+                if (!authid.equals(authzid)) {
+                  ac.setAuthorized(false);
+                } else {
+                  ac.setAuthorized(true);
+                  String userName = SecurityUtil.getUserFromPrincipal(authzid);
+                  LOG.info("Effective user: {}", userName);
+                  ac.setAuthorizedID(userName);
+                }
+              }
+            }
+          });
+      transportFactory = saslFactory;
+
+      // Create a processor wrapper, to get the caller
+      processorToUse = (inProt, outProt) -> {
+        TSaslServerTransport saslServerTransport =
+            (TSaslServerTransport)inProt.getTransport();
+        SaslServer saslServer = saslServerTransport.getSaslServer();
+        String principal = saslServer.getAuthorizationID();
+        hbaseServiceHandler.setEffectiveUser(principal);
+        return processor.process(inProt, outProt);
+      };
+    }
+
+    if (conf.get(BIND_CONF_KEY) != null && !implType.canSpecifyBindIP) {
+      LOG.error("Server types {} don't support IP address binding at the moment. See " +
+              "https://issues.apache.org/jira/browse/HBASE-2155 for details.",
+          Joiner.on(", ").join(ImplType.serversThatCannotSpecifyBindIP()));
+      throw new RuntimeException("-" + BIND_CONF_KEY + " not supported with " + implType);
+    }
+
+    InetSocketAddress inetSocketAddress = new InetSocketAddress(getBindAddress(conf), listenPort);
+    if (implType == ImplType.HS_HA || implType == ImplType.NONBLOCKING ||
+        implType == ImplType.THREADED_SELECTOR) {
+      TNonblockingServerTransport serverTransport = new TNonblockingServerSocket(inetSocketAddress);
+      if (implType == ImplType.NONBLOCKING) {
+        tserver = getTNonBlockingServer(serverTransport, protocolFactory, processorToUse,
+            transportFactory, inetSocketAddress);
+      } else if (implType == ImplType.HS_HA) {
+        tserver = getTHsHaServer(serverTransport, protocolFactory, processorToUse, transportFactory,
+            inetSocketAddress);
+      } else { // THREADED_SELECTOR
+        tserver = getTThreadedSelectorServer(serverTransport, protocolFactory, processorToUse,
+            transportFactory, inetSocketAddress);
+      }
+      LOG.info("starting HBase {} server on {}", implType.simpleClassName(),
+          Integer.toString(listenPort));
+    } else if (implType == ImplType.THREAD_POOL) {
+      this.tserver = getTThreadPoolServer(protocolFactory, processorToUse, transportFactory,
+          inetSocketAddress);
+    } else {
+      throw new AssertionError("Unsupported Thrift server implementation: " +
+          implType.simpleClassName());
+    }
+
+    // A sanity check that we instantiated the right type of server.
+    if (tserver.getClass() != implType.serverClass) {
+      throw new AssertionError("Expected to create Thrift server class " +
+          implType.serverClass.getName() + " but got " +
+          tserver.getClass().getName());
+    }
+  }
+
+  private TServer getTNonBlockingServer(TNonblockingServerTransport serverTransport,
+      TProtocolFactory protocolFactory, TProcessor processor, TTransportFactory transportFactory,
+      InetSocketAddress inetSocketAddress) {
+    LOG.info("starting HBase Nonblocking Thrift server on " + inetSocketAddress.toString());
+    TNonblockingServer.Args serverArgs = new TNonblockingServer.Args(serverTransport);
+    serverArgs.processor(processor);
+    serverArgs.transportFactory(transportFactory);
+    serverArgs.protocolFactory(protocolFactory);
+    return new TNonblockingServer(serverArgs);
+  }
+
+  private TServer getTHsHaServer(TNonblockingServerTransport serverTransport,
+      TProtocolFactory protocolFactory, TProcessor processor, TTransportFactory transportFactory,
+      InetSocketAddress inetSocketAddress) {
+    LOG.info("starting HBase HsHA Thrift server on " + inetSocketAddress.toString());
+    THsHaServer.Args serverArgs = new THsHaServer.Args(serverTransport);
+    int queueSize = conf.getInt(TBoundedThreadPoolServer.MAX_QUEUED_REQUESTS_CONF_KEY,
+        TBoundedThreadPoolServer.DEFAULT_MAX_QUEUED_REQUESTS);
+    CallQueue callQueue = new CallQueue(new LinkedBlockingQueue<>(queueSize), metrics);
+    int workerThread = conf.getInt(TBoundedThreadPoolServer.MAX_WORKER_THREADS_CONF_KEY,
+        serverArgs.getMaxWorkerThreads());
+    ExecutorService executorService = createExecutor(
+        callQueue, workerThread, workerThread);
+    serverArgs.executorService(executorService).processor(processor)
+        .transportFactory(transportFactory).protocolFactory(protocolFactory);
+    return new THsHaServer(serverArgs);
+  }
+
+  private TServer getTThreadedSelectorServer(TNonblockingServerTransport serverTransport,
+      TProtocolFactory protocolFactory, TProcessor processor, TTransportFactory transportFactory,
+      InetSocketAddress inetSocketAddress) {
+    LOG.info("starting HBase ThreadedSelector Thrift server on " + inetSocketAddress.toString());
+    TThreadedSelectorServer.Args serverArgs =
+        new HThreadedSelectorServerArgs(serverTransport, conf);
+    int queueSize = conf.getInt(TBoundedThreadPoolServer.MAX_QUEUED_REQUESTS_CONF_KEY,
+        TBoundedThreadPoolServer.DEFAULT_MAX_QUEUED_REQUESTS);
+    CallQueue callQueue = new CallQueue(new LinkedBlockingQueue<>(queueSize), metrics);
+    int workerThreads = conf.getInt(TBoundedThreadPoolServer.MAX_WORKER_THREADS_CONF_KEY,
+        serverArgs.getWorkerThreads());
+    int selectorThreads = conf.getInt(THRIFT_SELECTOR_NUM, serverArgs.getSelectorThreads());
+    serverArgs.selectorThreads(selectorThreads);
+    ExecutorService executorService = createExecutor(
+        callQueue, workerThreads, workerThreads);
+    serverArgs.executorService(executorService).processor(processor)
+        .transportFactory(transportFactory).protocolFactory(protocolFactory);
+    return new TThreadedSelectorServer(serverArgs);
+  }
+
+  private TServer getTThreadPoolServer(TProtocolFactory protocolFactory, TProcessor processor,
+      TTransportFactory transportFactory, InetSocketAddress inetSocketAddress) throws Exception {
+    LOG.info("starting HBase ThreadPool Thrift server on " + inetSocketAddress.toString());
+    // Thrift's implementation uses '0' as a placeholder for 'use the default.'
+    int backlog = conf.getInt(BACKLOG_CONF_KEY, BACKLOG_CONF_DEAFULT);
+    int readTimeout = conf.getInt(THRIFT_SERVER_SOCKET_READ_TIMEOUT_KEY,
+        THRIFT_SERVER_SOCKET_READ_TIMEOUT_DEFAULT);
+    TServerTransport serverTransport = new TServerSocket(
+        new TServerSocket.ServerSocketTransportArgs().
+            bindAddr(inetSocketAddress).backlog(backlog).
+            clientTimeout(readTimeout));
+
+    TBoundedThreadPoolServer.Args serverArgs =
+        new TBoundedThreadPoolServer.Args(serverTransport, conf);
+    serverArgs.processor(processor).transportFactory(transportFactory)
+        .protocolFactory(protocolFactory);
+    return new TBoundedThreadPoolServer(serverArgs, metrics);
+  }
+
+  private TProtocolFactory getProtocolFactory() {
+    TProtocolFactory protocolFactory;
+
+    if (conf.getBoolean(COMPACT_CONF_KEY, COMPACT_CONF_DEFAULT)) {
+      LOG.debug("Using compact protocol");
+      protocolFactory = new TCompactProtocol.Factory();
+    } else {
+      LOG.debug("Using binary protocol");
+      protocolFactory = new TBinaryProtocol.Factory();
+    }
+
+    return protocolFactory;
+  }
+
+  ExecutorService createExecutor(BlockingQueue<Runnable> callQueue,
+      int minWorkers, int maxWorkers) {
+    ThreadFactoryBuilder tfb = new ThreadFactoryBuilder();
+    tfb.setDaemon(true);
+    tfb.setNameFormat("thrift-worker-%d");
+    ThreadPoolExecutor threadPool = new THBaseThreadPoolExecutor(minWorkers, maxWorkers,
+        Long.MAX_VALUE, TimeUnit.SECONDS, callQueue, tfb.build(), metrics);
+    threadPool.allowCoreThreadTimeOut(true);
+    return threadPool;
+  }
+
+  private InetAddress getBindAddress(Configuration conf)
+      throws UnknownHostException {
+    String bindAddressStr = conf.get(BIND_CONF_KEY, DEFAULT_BIND_ADDR);
+    return InetAddress.getByName(bindAddressStr);
+  }
+
+
+  public static void registerFilters(Configuration conf) {
+    String[] filters = conf.getStrings(THRIFT_FILTERS);
+    Splitter splitter = Splitter.on(':');
+    if(filters != null) {
+      for(String filterClass: filters) {
+        List<String> filterPart = splitter.splitToList(filterClass);
+        if(filterPart.size() != 2) {
+          LOG.warn("Invalid filter specification " + filterClass + " - skipping");
+        } else {
+          ParseFilter.registerFilter(filterPart.get(0), filterPart.get(1));
+        }
+      }
+    }
+  }
+
+  /**
+   * Add options to command lines
+   * @param options options
+   */
+  protected void addOptions(Options options) {
     options.addOption("b", BIND_OPTION, true, "Address to bind " +
         "the Thrift server to. [default: " + DEFAULT_BIND_ADDR + "]");
     options.addOption("p", PORT_OPTION, true, "Port to bind to [default: " +
@@ -118,62 +624,56 @@ public class ThriftServer {
     options.addOption("f", FRAMED_OPTION, false, "Use framed transport");
     options.addOption("c", COMPACT_OPTION, false, "Use the compact protocol");
     options.addOption("h", "help", false, "Print help information");
+    options.addOption("s", SELECTOR_NUM_OPTION, true, "How many selector threads to use.");
     options.addOption(null, INFOPORT_OPTION, true, "Port for web UI");
 
     options.addOption("m", MIN_WORKERS_OPTION, true,
         "The minimum number of worker threads for " +
-        ImplType.THREAD_POOL.simpleClassName());
+            ImplType.THREAD_POOL.simpleClassName());
 
     options.addOption("w", MAX_WORKERS_OPTION, true,
         "The maximum number of worker threads for " +
-        ImplType.THREAD_POOL.simpleClassName());
+            ImplType.THREAD_POOL.simpleClassName());
 
     options.addOption("q", MAX_QUEUE_SIZE_OPTION, true,
         "The maximum number of queued requests in " +
-        ImplType.THREAD_POOL.simpleClassName());
+            ImplType.THREAD_POOL.simpleClassName());
 
     options.addOption("k", KEEP_ALIVE_SEC_OPTION, true,
         "The amount of time in secods to keep a thread alive when idle in " +
-        ImplType.THREAD_POOL.simpleClassName());
+            ImplType.THREAD_POOL.simpleClassName());
 
     options.addOption("t", READ_TIMEOUT_OPTION, true,
         "Amount of time in milliseconds before a server thread will timeout " +
-        "waiting for client to send data on a connected socket. Currently, " +
-        "only applies to TBoundedThreadPoolServer");
+            "waiting for client to send data on a connected socket. Currently, " +
+            "only applies to TBoundedThreadPoolServer");
 
     options.addOptionGroup(ImplType.createOptionGroup());
+  }
 
-    CommandLineParser parser = new DefaultParser();
-    CommandLine cmd = parser.parse(options, args);
-
-    if (cmd.hasOption("help")) {
-      printUsageAndExit(options, 1);
-    }
-
+  protected void parseCommandLine(CommandLine cmd, Options options) throws ExitCodeException {
     // Get port to bind to
     try {
       if (cmd.hasOption(PORT_OPTION)) {
         int listenPort = Integer.parseInt(cmd.getOptionValue(PORT_OPTION));
-        conf.setInt(ThriftServerRunner.PORT_CONF_KEY, listenPort);
+        conf.setInt(PORT_CONF_KEY, listenPort);
       }
     } catch (NumberFormatException e) {
       LOG.error("Could not parse the value provided for the port option", e);
       printUsageAndExit(options, -1);
     }
-
     // check for user-defined info server port setting, if so override the conf
     try {
       if (cmd.hasOption(INFOPORT_OPTION)) {
         String val = cmd.getOptionValue(INFOPORT_OPTION);
-        conf.setInt("hbase.thrift.info.port", Integer.parseInt(val));
+        conf.setInt(THRIFT_INFO_SERVER_PORT, Integer.parseInt(val));
         LOG.debug("Web UI port set to " + val);
       }
     } catch (NumberFormatException e) {
       LOG.error("Could not parse the value provided for the " + INFOPORT_OPTION +
-        " option", e);
+          " option", e);
       printUsageAndExit(options, -1);
     }
-
     // Make optional changes to the configuration based on command-line options
     optionToConf(cmd, MIN_WORKERS_OPTION,
         conf, TBoundedThreadPoolServer.MIN_WORKER_THREADS_CONF_KEY);
@@ -183,23 +683,42 @@ public class ThriftServer {
         conf, TBoundedThreadPoolServer.MAX_QUEUED_REQUESTS_CONF_KEY);
     optionToConf(cmd, KEEP_ALIVE_SEC_OPTION,
         conf, TBoundedThreadPoolServer.THREAD_KEEP_ALIVE_TIME_SEC_CONF_KEY);
-    optionToConf(cmd, READ_TIMEOUT_OPTION, conf,
-        ThriftServerRunner.THRIFT_SERVER_SOCKET_READ_TIMEOUT_KEY);
+    optionToConf(cmd, READ_TIMEOUT_OPTION, conf, THRIFT_SERVER_SOCKET_READ_TIMEOUT_KEY);
+    optionToConf(cmd, SELECTOR_NUM_OPTION, conf, THRIFT_SELECTOR_NUM);
 
     // Set general thrift server options
     boolean compact = cmd.hasOption(COMPACT_OPTION) ||
-      conf.getBoolean(ThriftServerRunner.COMPACT_CONF_KEY, false);
-    conf.setBoolean(ThriftServerRunner.COMPACT_CONF_KEY, compact);
+        conf.getBoolean(COMPACT_CONF_KEY, false);
+    conf.setBoolean(COMPACT_CONF_KEY, compact);
     boolean framed = cmd.hasOption(FRAMED_OPTION) ||
-      conf.getBoolean(ThriftServerRunner.FRAMED_CONF_KEY, false);
-    conf.setBoolean(ThriftServerRunner.FRAMED_CONF_KEY, framed);
-    if (cmd.hasOption(BIND_OPTION)) {
-      conf.set(ThriftServerRunner.BIND_CONF_KEY, cmd.getOptionValue(BIND_OPTION));
-    }
+        conf.getBoolean(FRAMED_CONF_KEY, false);
+    conf.setBoolean(FRAMED_CONF_KEY, framed);
+
+    optionToConf(cmd, BIND_OPTION, conf, BIND_CONF_KEY);
+
 
     ImplType.setServerImpl(cmd, conf);
   }
 
+  /**
+   * Parse the command line options to set parameters the conf.
+   */
+  private void processOptions(final String[] args) throws Exception {
+    if (args == null || args.length == 0) {
+      return;
+    }
+    Options options = new Options();
+    addOptions(options);
+
+    CommandLineParser parser = new DefaultParser();
+    CommandLine cmd = parser.parse(options, args);
+
+    if (cmd.hasOption("help")) {
+      printUsageAndExit(options, 1);
+    }
+    parseCommandLine(cmd, options);
+  }
+
   public void stop() {
     if (this.infoServer != null) {
       LOG.info("Stopping infoServer");
@@ -209,10 +728,25 @@ public class ThriftServer {
         LOG.error("Failed to stop infoServer", ex);
       }
     }
-    serverRunner.shutdown();
+    if (pauseMonitor != null) {
+      pauseMonitor.stop();
+    }
+    if (tserver != null) {
+      tserver.stop();
+      tserver = null;
+    }
+    if (httpServer != null) {
+      try {
+        httpServer.stop();
+        httpServer = null;
+      } catch (Exception e) {
+        LOG.error("Problem encountered in shutting down HTTP server", e);
+      }
+      httpServer = null;
+    }
   }
 
-  private static void optionToConf(CommandLine cmd, String option,
+  protected static void optionToConf(CommandLine cmd, String option,
       Configuration conf, String destConfKey) {
     if (cmd.hasOption(option)) {
       String value = cmd.getOptionValue(option);
@@ -221,16 +755,38 @@ public class ThriftServer {
     }
   }
 
+  /**
+   * Run without any command line arguments
+   * @return exit code
+   * @throws Exception exception
+   */
+  public int run() throws Exception {
+    return run(null);
+  }
+
+  @Override
+  public int run(String[] strings) throws Exception {
+    processOptions(strings);
+    setupParamters();
+    startInfoServer();
+    if (httpEnabled) {
+      setupHTTPServer();
+      httpServer.start();
+      httpServer.join();
+    } else {
+      setupServer();
+      tserver.serve();
+    }
+    return 0;
+  }
+
   public static void main(String [] args) throws Exception {
     LOG.info("***** STARTING service '" + ThriftServer.class.getSimpleName() + "' *****");
     VersionInfo.logVersion();
-    int exitCode = 0;
-    try {
-      new ThriftServer(HBaseConfiguration.create()).doMain(args);
-    } catch (ExitCodeException ex) {
-      exitCode = ex.getExitCode();
-    }
+    final Configuration conf = HBaseConfiguration.create();
+    // for now, only time we return is on an argument error.
+    final int status = ToolRunner.run(conf, new ThriftServer(conf), args);
     LOG.info("***** STOPPING service '" + ThriftServer.class.getSimpleName() + "' *****");
-    System.exit(exitCode);
+    System.exit(status);
   }
 }


[14/15] hbase git commit: HBASE-21526 Use AsyncClusterConnection in ServerManager for getRsAdmin

Posted by zh...@apache.org.
HBASE-21526 Use AsyncClusterConnection in ServerManager for getRsAdmin


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

Branch: refs/heads/HBASE-21512
Commit: 0d051abb71d2f8b0846c078ce7ce452c4372aa33
Parents: 721af37
Author: zhangduo <zh...@apache.org>
Authored: Thu Dec 6 21:25:34 2018 +0800
Committer: Duo Zhang <zh...@apache.org>
Committed: Fri Jan 4 14:34:02 2019 +0800

----------------------------------------------------------------------
 .../hbase/client/AsyncClusterConnection.java    |   6 +
 .../hbase/client/AsyncConnectionImpl.java       |   5 +
 .../hbase/client/AsyncRegionServerAdmin.java    | 210 +++++++++++++++++++
 .../apache/hadoop/hbase/util/FutureUtils.java   |   2 +-
 .../org/apache/hadoop/hbase/master/HMaster.java |  15 +-
 .../hadoop/hbase/master/ServerManager.java      |  67 ------
 .../master/procedure/RSProcedureDispatcher.java |  44 ++--
 7 files changed, 263 insertions(+), 86 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/0d051abb/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnection.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnection.java
index c7dea25..1327fd7 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnection.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnection.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hbase.client;
 
+import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.ipc.RpcClient;
 import org.apache.yetus.audience.InterfaceAudience;
 
@@ -27,6 +28,11 @@ import org.apache.yetus.audience.InterfaceAudience;
 public interface AsyncClusterConnection extends AsyncConnection {
 
   /**
+   * Get the admin service for the given region server.
+   */
+  AsyncRegionServerAdmin getRegionServerAdmin(ServerName serverName);
+
+  /**
    * Get the nonce generator for this connection.
    */
   NonceGenerator getNonceGenerator();

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d051abb/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
index 188e830..4e7f421 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
@@ -330,4 +330,9 @@ class AsyncConnectionImpl implements AsyncClusterConnection {
     return new AsyncBufferedMutatorBuilderImpl(connConf, getTableBuilder(tableName, pool),
       RETRY_TIMER);
   }
+
+  @Override
+  public AsyncRegionServerAdmin getRegionServerAdmin(ServerName serverName) {
+    return new AsyncRegionServerAdmin(serverName, this);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d051abb/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionServerAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionServerAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionServerAdmin.java
new file mode 100644
index 0000000..9accd89
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionServerAdmin.java
@@ -0,0 +1,210 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import java.io.IOException;
+import java.util.concurrent.CompletableFuture;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.ipc.HBaseRpcController;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearCompactionQueuesRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearCompactionQueuesResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearRegionBlockCacheRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearRegionBlockCacheResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactionSwitchRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactionSwitchResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ExecuteProceduresRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ExecuteProceduresResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionLoadRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionLoadResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse;
+
+/**
+ * A simple wrapper of the {@link AdminService} for a region server, which returns a
+ * {@link CompletableFuture}. This is easier to use, as if you use the raw protobuf interface, you
+ * need to get the result from the {@link RpcCallback}, and if there is an exception, you need to
+ * get it from the {@link RpcController} passed in.
+ * <p/>
+ * Notice that there is no retry, and this is intentional. We have different retry for different
+ * usage for now, if later we want to unify them, we can move the retry logic into this class.
+ */
+@InterfaceAudience.Private
+public class AsyncRegionServerAdmin {
+
+  private final ServerName server;
+
+  private final AsyncConnectionImpl conn;
+
+  AsyncRegionServerAdmin(ServerName server, AsyncConnectionImpl conn) {
+    this.server = server;
+    this.conn = conn;
+  }
+
+  @FunctionalInterface
+  private interface RpcCall<RESP> {
+    void call(AdminService.Interface stub, HBaseRpcController controller, RpcCallback<RESP> done);
+  }
+
+  private <RESP> CompletableFuture<RESP> call(RpcCall<RESP> rpcCall) {
+    CompletableFuture<RESP> future = new CompletableFuture<>();
+    HBaseRpcController controller = conn.rpcControllerFactory.newController();
+    try {
+      rpcCall.call(conn.getAdminStub(server), controller, new RpcCallback<RESP>() {
+
+        @Override
+        public void run(RESP resp) {
+          if (controller.failed()) {
+            future.completeExceptionally(controller.getFailed());
+          } else {
+            future.complete(resp);
+          }
+        }
+      });
+    } catch (IOException e) {
+      future.completeExceptionally(e);
+    }
+    return future;
+  }
+
+  public CompletableFuture<GetRegionInfoResponse> getRegionInfo(GetRegionInfoRequest request) {
+    return call((stub, controller, done) -> stub.getRegionInfo(controller, request, done));
+  }
+
+  public CompletableFuture<GetStoreFileResponse> getStoreFile(GetStoreFileRequest request) {
+    return call((stub, controller, done) -> stub.getStoreFile(controller, request, done));
+  }
+
+  public CompletableFuture<GetOnlineRegionResponse> getOnlineRegion(
+      GetOnlineRegionRequest request) {
+    return call((stub, controller, done) -> stub.getOnlineRegion(controller, request, done));
+  }
+
+  public CompletableFuture<OpenRegionResponse> openRegion(OpenRegionRequest request) {
+    return call((stub, controller, done) -> stub.openRegion(controller, request, done));
+  }
+
+  public CompletableFuture<WarmupRegionResponse> warmupRegion(WarmupRegionRequest request) {
+    return call((stub, controller, done) -> stub.warmupRegion(controller, request, done));
+  }
+
+  public CompletableFuture<CloseRegionResponse> closeRegion(CloseRegionRequest request) {
+    return call((stub, controller, done) -> stub.closeRegion(controller, request, done));
+  }
+
+  public CompletableFuture<FlushRegionResponse> flushRegion(FlushRegionRequest request) {
+    return call((stub, controller, done) -> stub.flushRegion(controller, request, done));
+  }
+
+  public CompletableFuture<CompactionSwitchResponse> compactionSwitch(
+      CompactionSwitchRequest request) {
+    return call((stub, controller, done) -> stub.compactionSwitch(controller, request, done));
+  }
+
+  public CompletableFuture<CompactRegionResponse> compactRegion(CompactRegionRequest request) {
+    return call((stub, controller, done) -> stub.compactRegion(controller, request, done));
+  }
+
+  public CompletableFuture<ReplicateWALEntryResponse> replicateWALEntry(
+      ReplicateWALEntryRequest request) {
+    return call((stub, controller, done) -> stub.replicateWALEntry(controller, request, done));
+  }
+
+  public CompletableFuture<ReplicateWALEntryResponse> replay(ReplicateWALEntryRequest request) {
+    return call((stub, controller, done) -> stub.replay(controller, request, done));
+  }
+
+  public CompletableFuture<RollWALWriterResponse> rollWALWriter(RollWALWriterRequest request) {
+    return call((stub, controller, done) -> stub.rollWALWriter(controller, request, done));
+  }
+
+  public CompletableFuture<GetServerInfoResponse> getServerInfo(GetServerInfoRequest request) {
+    return call((stub, controller, done) -> stub.getServerInfo(controller, request, done));
+  }
+
+  public CompletableFuture<StopServerResponse> stopServer(StopServerRequest request) {
+    return call((stub, controller, done) -> stub.stopServer(controller, request, done));
+  }
+
+  public CompletableFuture<UpdateFavoredNodesResponse> updateFavoredNodes(
+      UpdateFavoredNodesRequest request) {
+    return call((stub, controller, done) -> stub.updateFavoredNodes(controller, request, done));
+  }
+
+  public CompletableFuture<UpdateConfigurationResponse> updateConfiguration(
+      UpdateConfigurationRequest request) {
+    return call((stub, controller, done) -> stub.updateConfiguration(controller, request, done));
+  }
+
+  public CompletableFuture<GetRegionLoadResponse> getRegionLoad(GetRegionLoadRequest request) {
+    return call((stub, controller, done) -> stub.getRegionLoad(controller, request, done));
+  }
+
+  public CompletableFuture<ClearCompactionQueuesResponse> clearCompactionQueues(
+      ClearCompactionQueuesRequest request) {
+    return call((stub, controller, done) -> stub.clearCompactionQueues(controller, request, done));
+  }
+
+  public CompletableFuture<ClearRegionBlockCacheResponse> clearRegionBlockCache(
+      ClearRegionBlockCacheRequest request) {
+    return call((stub, controller, done) -> stub.clearRegionBlockCache(controller, request, done));
+  }
+
+  public CompletableFuture<GetSpaceQuotaSnapshotsResponse> getSpaceQuotaSnapshots(
+      GetSpaceQuotaSnapshotsRequest request) {
+    return call((stub, controller, done) -> stub.getSpaceQuotaSnapshots(controller, request, done));
+  }
+
+  public CompletableFuture<ExecuteProceduresResponse> executeProcedures(
+      ExecuteProceduresRequest request) {
+    return call((stub, controller, done) -> stub.executeProcedures(controller, request, done));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d051abb/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FutureUtils.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FutureUtils.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FutureUtils.java
index 067e66b..f4a7332 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FutureUtils.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FutureUtils.java
@@ -57,4 +57,4 @@ public final class FutureUtils {
       }
     });
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d051abb/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index 52005d6..4fd3a37 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -193,6 +193,7 @@ import org.apache.hadoop.hbase.util.BloomFilterUtil;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.CompressionTest;
 import org.apache.hadoop.hbase.util.EncryptionTest;
+import org.apache.hadoop.hbase.util.FutureUtils;
 import org.apache.hadoop.hbase.util.HBaseFsck;
 import org.apache.hadoop.hbase.util.HFileArchiveUtil;
 import org.apache.hadoop.hbase.util.HasThread;
@@ -225,6 +226,7 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
 
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceViolationPolicy;
@@ -1937,6 +1939,15 @@ public class HMaster extends HRegionServer implements MasterServices {
     });
   }
 
+  private void warmUpRegion(ServerName server, RegionInfo region) {
+    FutureUtils.addListener(asyncClusterConnection.getRegionServerAdmin(server)
+      .warmupRegion(RequestConverter.buildWarmupRegionRequest(region)), (r, e) -> {
+        if (e != null) {
+          LOG.warn("Failed to warm up region {} on server {}", region, server, e);
+        }
+      });
+  }
+
   // Public so can be accessed by tests. Blocks until move is done.
   // Replace with an async implementation from which you can get
   // a success/failure result.
@@ -2008,7 +2019,9 @@ public class HMaster extends HRegionServer implements MasterServices {
       // Warmup the region on the destination before initiating the move. this call
       // is synchronous and takes some time. doing it before the source region gets
       // closed
-      serverManager.sendRegionWarmup(rp.getDestination(), hri);
+      // A region server could reject the close request because it either does not
+      // have the specified region or the region is being split.
+      warmUpRegion(rp.getDestination(), hri);
 
       LOG.info(getClientIdAuditPrefix() + " move " + rp + ", running balancer");
       Future<byte []> future = this.assignmentManager.moveAsync(rp);

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d051abb/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 86d72d1..c26ef6c 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
@@ -24,7 +24,6 @@ import java.io.IOException;
 import java.net.InetAddress;
 import java.util.ArrayList;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
@@ -51,12 +50,9 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.YouAreDeadException;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.RegionInfo;
-import org.apache.hadoop.hbase.client.RetriesExhaustedException;
 import org.apache.hadoop.hbase.ipc.HBaseRpcController;
-import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.master.assignment.RegionStates;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
-import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
@@ -159,25 +155,16 @@ public class ServerManager {
   private final ConcurrentNavigableMap<ServerName, ServerMetrics> onlineServers =
     new ConcurrentSkipListMap<>();
 
-  /**
-   * Map of admin interfaces per registered regionserver; these interfaces we use to control
-   * regionservers out on the cluster
-   */
-  private final Map<ServerName, AdminService.BlockingInterface> rsAdmins = new HashMap<>();
-
   /** List of region servers that should not get any more new regions. */
   private final ArrayList<ServerName> drainingServers = new ArrayList<>();
 
   private final MasterServices master;
-  private final ClusterConnection connection;
 
   private final DeadServer deadservers = new DeadServer();
 
   private final long maxSkew;
   private final long warningSkew;
 
-  private final RpcControllerFactory rpcControllerFactory;
-
   /** Listeners that are called on server events. */
   private List<ServerListener> listeners = new CopyOnWriteArrayList<>();
 
@@ -189,8 +176,6 @@ public class ServerManager {
     Configuration c = master.getConfiguration();
     maxSkew = c.getLong("hbase.master.maxclockskew", 30000);
     warningSkew = c.getLong("hbase.master.warningclockskew", 10000);
-    this.connection = master.getClusterConnection();
-    this.rpcControllerFactory = this.connection == null? null: connection.getRpcControllerFactory();
     persistFlushedSequenceId = c.getBoolean(PERSIST_FLUSHEDSEQUENCEID,
         PERSIST_FLUSHEDSEQUENCEID_DEFAULT);
   }
@@ -438,7 +423,6 @@ public class ServerManager {
   void recordNewServerWithLock(final ServerName serverName, final ServerMetrics sl) {
     LOG.info("Registering regionserver=" + serverName);
     this.onlineServers.put(serverName, sl);
-    this.rsAdmins.remove(serverName);
   }
 
   @VisibleForTesting
@@ -633,7 +617,6 @@ public class ServerManager {
       this.onlineServers.remove(sn);
       onlineServers.notifyAll();
     }
-    this.rsAdmins.remove(sn);
   }
 
   /*
@@ -676,34 +659,6 @@ public class ServerManager {
     return this.drainingServers.add(sn);
   }
 
-  // RPC methods to region servers
-
-  private HBaseRpcController newRpcController() {
-    return rpcControllerFactory == null ? null : rpcControllerFactory.newController();
-  }
-
-  /**
-   * Sends a WARMUP RPC to the specified server to warmup the specified region.
-   * <p>
-   * A region server could reject the close request because it either does not
-   * have the specified region or the region is being split.
-   * @param server server to warmup a region
-   * @param region region to  warmup
-   */
-  public void sendRegionWarmup(ServerName server,
-      RegionInfo region) {
-    if (server == null) return;
-    try {
-      AdminService.BlockingInterface admin = getRsAdmin(server);
-      HBaseRpcController controller = newRpcController();
-      ProtobufUtil.warmupRegion(controller, admin, region);
-    } catch (IOException e) {
-      LOG.error("Received exception in RPC for warmup server:" +
-        server + "region: " + region +
-        "exception: " + e);
-    }
-  }
-
   /**
    * Contacts a region server and waits up to timeout ms
    * to close the region.  This bypasses the active hmaster.
@@ -737,28 +692,6 @@ public class ServerManager {
   }
 
   /**
-   * @param sn
-   * @return Admin interface for the remote regionserver named <code>sn</code>
-   * @throws IOException
-   * @throws RetriesExhaustedException wrapping a ConnectException if failed
-   */
-  public AdminService.BlockingInterface getRsAdmin(final ServerName sn)
-  throws IOException {
-    AdminService.BlockingInterface admin = this.rsAdmins.get(sn);
-    if (admin == null) {
-      LOG.debug("New admin connection to " + sn.toString());
-      if (sn.equals(master.getServerName()) && master instanceof HRegionServer) {
-        // A master is also a region server now, see HBASE-10569 for details
-        admin = ((HRegionServer)master).getRSRpcServices();
-      } else {
-        admin = this.connection.getAdmin(sn);
-      }
-      this.rsAdmins.put(sn, admin);
-    }
-    return admin;
-  }
-
-  /**
    * Calculate min necessary to start. This is not an absolute. It is just
    * a friction that will cause us hang around a bit longer waiting on
    * RegionServers to check-in.

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d051abb/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
index 638f9d3..f3ab4b3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
@@ -18,12 +18,15 @@
 package org.apache.hadoop.hbase.master.procedure;
 
 import java.io.IOException;
+import java.io.InterruptedIOException;
 import java.util.List;
 import java.util.Set;
 import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.AsyncRegionServerAdmin;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.exceptions.ClientExceptionsUtil;
 import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
@@ -37,11 +40,11 @@ import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hbase.thirdparty.com.google.common.base.Throwables;
 import org.apache.hbase.thirdparty.com.google.common.collect.ArrayListMultimap;
 import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
 import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
 import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
-import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
 
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
@@ -159,13 +162,8 @@ public class RSProcedureDispatcher
       this.serverName = serverName;
     }
 
-    protected AdminService.BlockingInterface getRsAdmin() throws IOException {
-      final AdminService.BlockingInterface admin = master.getServerManager().getRsAdmin(serverName);
-      if (admin == null) {
-        throw new IOException("Attempting to send OPEN RPC to server " + getServerName() +
-          " failed because no RPC connection found to this server");
-      }
-      return admin;
+    protected AsyncRegionServerAdmin getRsAdmin() throws IOException {
+      return master.getAsyncClusterConnection().getRegionServerAdmin(serverName);
     }
 
     protected ServerName getServerName() {
@@ -344,9 +342,13 @@ public class RSProcedureDispatcher
     protected ExecuteProceduresResponse sendRequest(final ServerName serverName,
         final ExecuteProceduresRequest request) throws IOException {
       try {
-        return getRsAdmin().executeProcedures(null, request);
-      } catch (ServiceException se) {
-        throw ProtobufUtil.getRemoteException(se);
+        return getRsAdmin().executeProcedures(request).get();
+      } catch (InterruptedException e) {
+        throw (IOException) new InterruptedIOException().initCause(e);
+      } catch (ExecutionException e) {
+        Throwable cause = e.getCause();
+        Throwables.propagateIfPossible(cause, IOException.class);
+        throw new IOException(cause);
       }
     }
 
@@ -407,9 +409,13 @@ public class RSProcedureDispatcher
     private OpenRegionResponse sendRequest(final ServerName serverName,
         final OpenRegionRequest request) throws IOException {
       try {
-        return getRsAdmin().openRegion(null, request);
-      } catch (ServiceException se) {
-        throw ProtobufUtil.getRemoteException(se);
+        return getRsAdmin().openRegion(request).get();
+      } catch (InterruptedException e) {
+        throw (IOException) new InterruptedIOException().initCause(e);
+      } catch (ExecutionException e) {
+        Throwable cause = e.getCause();
+        Throwables.propagateIfPossible(cause, IOException.class);
+        throw new IOException(cause);
       }
     }
 
@@ -453,9 +459,13 @@ public class RSProcedureDispatcher
     private CloseRegionResponse sendRequest(final ServerName serverName,
         final CloseRegionRequest request) throws IOException {
       try {
-        return getRsAdmin().closeRegion(null, request);
-      } catch (ServiceException se) {
-        throw ProtobufUtil.getRemoteException(se);
+        return getRsAdmin().closeRegion(request).get();
+      } catch (InterruptedException e) {
+        throw (IOException) new InterruptedIOException().initCause(e);
+      } catch (ExecutionException e) {
+        Throwable cause = e.getCause();
+        Throwables.propagateIfPossible(cause, IOException.class);
+        throw new IOException(cause);
       }
     }
 


[08/15] hbase git commit: HBASE-17356 Add replica get support

Posted by zh...@apache.org.
HBASE-17356 Add replica get support


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

Branch: refs/heads/HBASE-21512
Commit: db66e6cc9e1c6ea027631388aba688cb623b7d0a
Parents: e4b6b4a
Author: zhangduo <zh...@apache.org>
Authored: Tue Jan 1 21:59:37 2019 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu Jan 3 08:38:20 2019 +0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/RegionLocations.java    |   30 +-
 .../client/AsyncBatchRpcRetryingCaller.java     |  114 +-
 .../client/AsyncConnectionConfiguration.java    |   12 +
 .../hbase/client/AsyncConnectionImpl.java       |    1 -
 .../hbase/client/AsyncMetaRegionLocator.java    |  125 +-
 .../hbase/client/AsyncNonMetaRegionLocator.java |  291 +--
 .../hadoop/hbase/client/AsyncRegionLocator.java |  129 +-
 .../hbase/client/AsyncRegionLocatorHelper.java  |  147 ++
 .../hbase/client/AsyncRpcRetryingCaller.java    |   15 +-
 .../client/AsyncRpcRetryingCallerFactory.java   |   55 +-
 .../AsyncSingleRequestRpcRetryingCaller.java    |   71 +-
 .../hbase/client/AsyncTableRegionLocator.java   |   28 +-
 .../client/AsyncTableRegionLocatorImpl.java     |    6 +-
 .../hbase/client/ConnectionConfiguration.java   |    5 +-
 .../hadoop/hbase/client/RawAsyncHBaseAdmin.java | 2033 +++++++++---------
 .../hadoop/hbase/client/RawAsyncTableImpl.java  |  208 +-
 .../apache/hadoop/hbase/util/FutureUtils.java   |   60 +
 .../hbase/client/RegionReplicaTestHelper.java   |  161 ++
 .../client/TestAsyncMetaRegionLocator.java      |   55 +-
 .../client/TestAsyncNonMetaRegionLocator.java   |  126 +-
 ...syncNonMetaRegionLocatorConcurrenyLimit.java |   20 +-
 ...TestAsyncSingleRequestRpcRetryingCaller.java |   56 +-
 .../client/TestAsyncTableLocatePrefetch.java    |    4 +-
 .../client/TestAsyncTableRegionReplicasGet.java |  204 ++
 .../hbase/client/TestZKAsyncRegistry.java       |   44 +-
 25 files changed, 2366 insertions(+), 1634 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/db66e6cc/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionLocations.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionLocations.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionLocations.java
index fd6f3c7..f98bf03 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionLocations.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionLocations.java
@@ -56,8 +56,8 @@ public class RegionLocations {
     int index = 0;
     for (HRegionLocation loc : locations) {
       if (loc != null) {
-        if (loc.getRegionInfo().getReplicaId() >= maxReplicaId) {
-          maxReplicaId = loc.getRegionInfo().getReplicaId();
+        if (loc.getRegion().getReplicaId() >= maxReplicaId) {
+          maxReplicaId = loc.getRegion().getReplicaId();
           maxReplicaIdIndex = index;
         }
       }
@@ -72,7 +72,7 @@ public class RegionLocations {
       this.locations = new HRegionLocation[maxReplicaId + 1];
       for (HRegionLocation loc : locations) {
         if (loc != null) {
-          this.locations[loc.getRegionInfo().getReplicaId()] = loc;
+          this.locations[loc.getRegion().getReplicaId()] = loc;
         }
       }
     }
@@ -146,7 +146,7 @@ public class RegionLocations {
   public RegionLocations remove(HRegionLocation location) {
     if (location == null) return this;
     if (location.getRegion() == null) return this;
-    int replicaId = location.getRegionInfo().getReplicaId();
+    int replicaId = location.getRegion().getReplicaId();
     if (replicaId >= locations.length) return this;
 
     // check whether something to remove. HRL.compareTo() compares ONLY the
@@ -203,14 +203,14 @@ public class RegionLocations {
     // in case of region replication going down, we might have a leak here.
     int max = other.locations.length;
 
-    HRegionInfo regionInfo = null;
+    RegionInfo regionInfo = null;
     for (int i = 0; i < max; i++) {
       HRegionLocation thisLoc = this.getRegionLocation(i);
       HRegionLocation otherLoc = other.getRegionLocation(i);
-      if (regionInfo == null && otherLoc != null && otherLoc.getRegionInfo() != null) {
+      if (regionInfo == null && otherLoc != null && otherLoc.getRegion() != null) {
         // regionInfo is the first non-null HRI from other RegionLocations. We use it to ensure that
         // all replica region infos belong to the same region with same region id.
-        regionInfo = otherLoc.getRegionInfo();
+        regionInfo = otherLoc.getRegion();
       }
 
       HRegionLocation selectedLoc = selectRegionLocation(thisLoc,
@@ -232,7 +232,7 @@ public class RegionLocations {
       for (int i=0; i < newLocations.length; i++) {
         if (newLocations[i] != null) {
           if (!RegionReplicaUtil.isReplicasForSameRegion(regionInfo,
-            newLocations[i].getRegionInfo())) {
+            newLocations[i].getRegion())) {
             newLocations[i] = null;
           }
         }
@@ -273,9 +273,9 @@ public class RegionLocations {
       boolean checkForEquals, boolean force) {
     assert location != null;
 
-    int replicaId = location.getRegionInfo().getReplicaId();
+    int replicaId = location.getRegion().getReplicaId();
 
-    HRegionLocation oldLoc = getRegionLocation(location.getRegionInfo().getReplicaId());
+    HRegionLocation oldLoc = getRegionLocation(location.getRegion().getReplicaId());
     HRegionLocation selectedLoc = selectRegionLocation(oldLoc, location,
       checkForEquals, force);
 
@@ -288,8 +288,8 @@ public class RegionLocations {
     // ensure that all replicas share the same start code. Otherwise delete them
     for (int i=0; i < newLocations.length; i++) {
       if (newLocations[i] != null) {
-        if (!RegionReplicaUtil.isReplicasForSameRegion(location.getRegionInfo(),
-          newLocations[i].getRegionInfo())) {
+        if (!RegionReplicaUtil.isReplicasForSameRegion(location.getRegion(),
+          newLocations[i].getRegion())) {
           newLocations[i] = null;
         }
       }
@@ -317,8 +317,8 @@ public class RegionLocations {
   public HRegionLocation getRegionLocationByRegionName(byte[] regionName) {
     for (HRegionLocation loc : locations) {
       if (loc != null) {
-        if (Bytes.equals(loc.getRegionInfo().getRegionName(), regionName)
-            || Bytes.equals(loc.getRegionInfo().getEncodedNameAsBytes(), regionName)) {
+        if (Bytes.equals(loc.getRegion().getRegionName(), regionName)
+            || Bytes.equals(loc.getRegion().getEncodedNameAsBytes(), regionName)) {
           return loc;
         }
       }
@@ -331,7 +331,7 @@ public class RegionLocations {
   }
 
   public HRegionLocation getDefaultRegionLocation() {
-    return locations[HRegionInfo.DEFAULT_REPLICA_ID];
+    return locations[RegionInfo.DEFAULT_REPLICA_ID];
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/db66e6cc/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBatchRpcRetryingCaller.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBatchRpcRetryingCaller.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBatchRpcRetryingCaller.java
index 51b89a9..e268b2e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBatchRpcRetryingCaller.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBatchRpcRetryingCaller.java
@@ -23,8 +23,7 @@ import static org.apache.hadoop.hbase.client.ConnectionUtils.getPauseTime;
 import static org.apache.hadoop.hbase.client.ConnectionUtils.resetController;
 import static org.apache.hadoop.hbase.client.ConnectionUtils.translateException;
 import static org.apache.hadoop.hbase.util.CollectionUtils.computeIfAbsent;
-
-import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer;
+import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -43,24 +42,26 @@ import java.util.concurrent.TimeUnit;
 import java.util.function.Supplier;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
-
 import org.apache.hadoop.hbase.CellScannable;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hbase.client.MultiResponse.RegionResult;
 import org.apache.hadoop.hbase.client.RetriesExhaustedException.ThrowableWithExtraContext;
 import org.apache.hadoop.hbase.ipc.HBaseRpcController;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer;
+
 import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 
 /**
  * Retry caller for batch.
@@ -121,10 +122,10 @@ class AsyncBatchRpcRetryingCaller<T> {
   private static final class ServerRequest {
 
     public final ConcurrentMap<byte[], RegionRequest> actionsByRegion =
-        new ConcurrentSkipListMap<>(Bytes.BYTES_COMPARATOR);
+      new ConcurrentSkipListMap<>(Bytes.BYTES_COMPARATOR);
 
     public void addAction(HRegionLocation loc, Action action) {
-      computeIfAbsent(actionsByRegion, loc.getRegionInfo().getRegionName(),
+      computeIfAbsent(actionsByRegion, loc.getRegion().getRegionName(),
         () -> new RegionRequest(loc)).actions.add(action);
     }
   }
@@ -173,11 +174,10 @@ class AsyncBatchRpcRetryingCaller<T> {
       Throwable error, ServerName serverName) {
     if (tries > startLogErrorsCnt) {
       String regions =
-          regionsSupplier.get().map(r -> "'" + r.loc.getRegionInfo().getRegionNameAsString() + "'")
-              .collect(Collectors.joining(",", "[", "]"));
-      LOG.warn("Process batch for " + regions + " in " + tableName + " from " + serverName
-          + " failed, tries=" + tries,
-        error);
+        regionsSupplier.get().map(r -> "'" + r.loc.getRegion().getRegionNameAsString() + "'")
+          .collect(Collectors.joining(",", "[", "]"));
+      LOG.warn("Process batch for " + regions + " in " + tableName + " from " + serverName +
+        " failed, tries=" + tries, error);
     }
   }
 
@@ -191,7 +191,7 @@ class AsyncBatchRpcRetryingCaller<T> {
       errors = action2Errors.computeIfAbsent(action, k -> new ArrayList<>());
     }
     errors.add(new ThrowableWithExtraContext(error, EnvironmentEdgeManager.currentTime(),
-        getExtraContextForError(serverName)));
+      getExtraContextForError(serverName)));
   }
 
   private void addError(Iterable<Action> actions, Throwable error, ServerName serverName) {
@@ -204,7 +204,7 @@ class AsyncBatchRpcRetryingCaller<T> {
       return;
     }
     ThrowableWithExtraContext errorWithCtx =
-        new ThrowableWithExtraContext(error, currentTime, extras);
+      new ThrowableWithExtraContext(error, currentTime, extras);
     List<ThrowableWithExtraContext> errors = removeErrors(action);
     if (errors == null) {
       errors = Collections.singletonList(errorWithCtx);
@@ -227,7 +227,7 @@ class AsyncBatchRpcRetryingCaller<T> {
         return;
       }
       future.completeExceptionally(new RetriesExhaustedException(tries,
-          Optional.ofNullable(removeErrors(action)).orElse(Collections.emptyList())));
+        Optional.ofNullable(removeErrors(action)).orElse(Collections.emptyList())));
     });
   }
 
@@ -242,9 +242,9 @@ class AsyncBatchRpcRetryingCaller<T> {
       // multiRequestBuilder will be populated with region actions.
       // rowMutationsIndexMap will be non-empty after the call if there is RowMutations in the
       // action list.
-      RequestConverter.buildNoDataRegionActions(entry.getKey(),
-        entry.getValue().actions, cells, multiRequestBuilder, regionActionBuilder, actionBuilder,
-        mutationBuilder, nonceGroup, rowMutationsIndexMap);
+      RequestConverter.buildNoDataRegionActions(entry.getKey(), entry.getValue().actions, cells,
+        multiRequestBuilder, regionActionBuilder, actionBuilder, mutationBuilder, nonceGroup,
+        rowMutationsIndexMap);
     }
     return multiRequestBuilder.build();
   }
@@ -254,15 +254,15 @@ class AsyncBatchRpcRetryingCaller<T> {
       RegionResult regionResult, List<Action> failedActions, Throwable regionException) {
     Object result = regionResult.result.getOrDefault(action.getOriginalIndex(), regionException);
     if (result == null) {
-      LOG.error("Server " + serverName + " sent us neither result nor exception for row '"
-          + Bytes.toStringBinary(action.getAction().getRow()) + "' of "
-          + regionReq.loc.getRegionInfo().getRegionNameAsString());
+      LOG.error("Server " + serverName + " sent us neither result nor exception for row '" +
+        Bytes.toStringBinary(action.getAction().getRow()) + "' of " +
+        regionReq.loc.getRegion().getRegionNameAsString());
       addError(action, new RuntimeException("Invalid response"), serverName);
       failedActions.add(action);
     } else if (result instanceof Throwable) {
       Throwable error = translateException((Throwable) result);
       logException(tries, () -> Stream.of(regionReq), error, serverName);
-      conn.getLocator().updateCachedLocation(regionReq.loc, error);
+      conn.getLocator().updateCachedLocationOnError(regionReq.loc, error);
       if (error instanceof DoNotRetryIOException || tries >= maxAttempts) {
         failOne(action, tries, error, EnvironmentEdgeManager.currentTime(),
           getExtraContextForError(serverName));
@@ -281,20 +281,19 @@ class AsyncBatchRpcRetryingCaller<T> {
       RegionResult regionResult = resp.getResults().get(rn);
       Throwable regionException = resp.getException(rn);
       if (regionResult != null) {
-        regionReq.actions.forEach(
-          action -> onComplete(action, regionReq, tries, serverName, regionResult, failedActions,
-            regionException));
+        regionReq.actions.forEach(action -> onComplete(action, regionReq, tries, serverName,
+          regionResult, failedActions, regionException));
       } else {
         Throwable error;
         if (regionException == null) {
-          LOG.error(
-            "Server sent us neither results nor exceptions for " + Bytes.toStringBinary(rn));
+          LOG
+            .error("Server sent us neither results nor exceptions for " + Bytes.toStringBinary(rn));
           error = new RuntimeException("Invalid response");
         } else {
           error = translateException(regionException);
         }
         logException(tries, () -> Stream.of(regionReq), error, serverName);
-        conn.getLocator().updateCachedLocation(regionReq.loc, error);
+        conn.getLocator().updateCachedLocationOnError(regionReq.loc, error);
         if (error instanceof DoNotRetryIOException || tries >= maxAttempts) {
           failAll(regionReq.actions.stream(), tries, error, serverName);
           return;
@@ -314,8 +313,7 @@ class AsyncBatchRpcRetryingCaller<T> {
       remainingNs = remainingTimeNs();
       if (remainingNs <= 0) {
         failAll(actionsByServer.values().stream().flatMap(m -> m.actionsByRegion.values().stream())
-            .flatMap(r -> r.actions.stream()),
-          tries);
+          .flatMap(r -> r.actions.stream()), tries);
         return;
       }
     } else {
@@ -366,15 +364,15 @@ class AsyncBatchRpcRetryingCaller<T> {
       ServerName serverName) {
     Throwable error = translateException(t);
     logException(tries, () -> actionsByRegion.values().stream(), error, serverName);
-    actionsByRegion
-        .forEach((rn, regionReq) -> conn.getLocator().updateCachedLocation(regionReq.loc, error));
+    actionsByRegion.forEach(
+      (rn, regionReq) -> conn.getLocator().updateCachedLocationOnError(regionReq.loc, error));
     if (error instanceof DoNotRetryIOException || tries >= maxAttempts) {
       failAll(actionsByRegion.values().stream().flatMap(r -> r.actions.stream()), tries, error,
         serverName);
       return;
     }
     List<Action> copiedActions = actionsByRegion.values().stream().flatMap(r -> r.actions.stream())
-        .collect(Collectors.toList());
+      .collect(Collectors.toList());
     addError(copiedActions, error, serverName);
     tryResubmit(copiedActions.stream(), tries);
   }
@@ -407,30 +405,30 @@ class AsyncBatchRpcRetryingCaller<T> {
     }
     ConcurrentMap<ServerName, ServerRequest> actionsByServer = new ConcurrentHashMap<>();
     ConcurrentLinkedQueue<Action> locateFailed = new ConcurrentLinkedQueue<>();
-    CompletableFuture.allOf(actions
-        .map(action -> conn.getLocator().getRegionLocation(tableName, action.getAction().getRow(),
-          RegionLocateType.CURRENT, locateTimeoutNs).whenComplete((loc, error) -> {
-            if (error != null) {
-              error = translateException(error);
-              if (error instanceof DoNotRetryIOException) {
-                failOne(action, tries, error, EnvironmentEdgeManager.currentTime(), "");
-                return;
-              }
-              addError(action, error, null);
-              locateFailed.add(action);
-            } else {
-              computeIfAbsent(actionsByServer, loc.getServerName(), ServerRequest::new)
-                  .addAction(loc, action);
+    addListener(CompletableFuture.allOf(actions
+      .map(action -> conn.getLocator().getRegionLocation(tableName, action.getAction().getRow(),
+        RegionLocateType.CURRENT, locateTimeoutNs).whenComplete((loc, error) -> {
+          if (error != null) {
+            error = translateException(error);
+            if (error instanceof DoNotRetryIOException) {
+              failOne(action, tries, error, EnvironmentEdgeManager.currentTime(), "");
+              return;
             }
-          }))
-        .toArray(CompletableFuture[]::new)).whenComplete((v, r) -> {
-          if (!actionsByServer.isEmpty()) {
-            send(actionsByServer, tries);
-          }
-          if (!locateFailed.isEmpty()) {
-            tryResubmit(locateFailed.stream(), tries);
+            addError(action, error, null);
+            locateFailed.add(action);
+          } else {
+            computeIfAbsent(actionsByServer, loc.getServerName(), ServerRequest::new).addAction(loc,
+              action);
           }
-        });
+        }))
+      .toArray(CompletableFuture[]::new)), (v, r) -> {
+        if (!actionsByServer.isEmpty()) {
+          send(actionsByServer, tries);
+        }
+        if (!locateFailed.isEmpty()) {
+          tryResubmit(locateFailed.stream(), tries);
+        }
+      });
   }
 
   public List<CompletableFuture<T>> call() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/db66e6cc/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionConfiguration.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionConfiguration.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionConfiguration.java
index 915e9dd..fa051a5 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionConfiguration.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionConfiguration.java
@@ -39,6 +39,8 @@ import static org.apache.hadoop.hbase.HConstants.HBASE_RPC_TIMEOUT_KEY;
 import static org.apache.hadoop.hbase.HConstants.HBASE_RPC_WRITE_TIMEOUT_KEY;
 import static org.apache.hadoop.hbase.client.AsyncProcess.DEFAULT_START_LOG_ERRORS_AFTER_COUNT;
 import static org.apache.hadoop.hbase.client.AsyncProcess.START_LOG_ERRORS_AFTER_COUNT_KEY;
+import static org.apache.hadoop.hbase.client.ConnectionConfiguration.PRIMARY_CALL_TIMEOUT_MICROSECOND;
+import static org.apache.hadoop.hbase.client.ConnectionConfiguration.PRIMARY_CALL_TIMEOUT_MICROSECOND_DEFAULT;
 import static org.apache.hadoop.hbase.client.ConnectionConfiguration.WRITE_BUFFER_PERIODIC_FLUSH_TIMEOUT_MS;
 import static org.apache.hadoop.hbase.client.ConnectionConfiguration.WRITE_BUFFER_PERIODIC_FLUSH_TIMEOUT_MS_DEFAULT;
 import static org.apache.hadoop.hbase.client.ConnectionConfiguration.WRITE_BUFFER_SIZE_DEFAULT;
@@ -94,6 +96,10 @@ class AsyncConnectionConfiguration {
 
   private final long writeBufferPeriodicFlushTimeoutNs;
 
+  // this is for supporting region replica get, if the primary does not finished within this
+  // timeout, we will send request to secondaries.
+  private final long primaryCallTimeoutNs;
+
   @SuppressWarnings("deprecation")
   AsyncConnectionConfiguration(Configuration conf) {
     this.metaOperationTimeoutNs = TimeUnit.MILLISECONDS.toNanos(
@@ -124,6 +130,8 @@ class AsyncConnectionConfiguration {
     this.writeBufferPeriodicFlushTimeoutNs =
       TimeUnit.MILLISECONDS.toNanos(conf.getLong(WRITE_BUFFER_PERIODIC_FLUSH_TIMEOUT_MS,
         WRITE_BUFFER_PERIODIC_FLUSH_TIMEOUT_MS_DEFAULT));
+    this.primaryCallTimeoutNs = TimeUnit.MICROSECONDS.toNanos(
+      conf.getLong(PRIMARY_CALL_TIMEOUT_MICROSECOND, PRIMARY_CALL_TIMEOUT_MICROSECOND_DEFAULT));
   }
 
   long getMetaOperationTimeoutNs() {
@@ -181,4 +189,8 @@ class AsyncConnectionConfiguration {
   long getWriteBufferPeriodicFlushTimeoutNs() {
     return writeBufferPeriodicFlushTimeoutNs;
   }
+
+  long getPrimaryCallTimeoutNs() {
+    return primaryCallTimeoutNs;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/db66e6cc/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
index 078395b..361d5b2 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
@@ -152,7 +152,6 @@ class AsyncConnectionImpl implements AsyncConnection {
   }
 
   // we will override this method for testing retry caller, so do not remove this method.
-  @VisibleForTesting
   AsyncRegionLocator getLocator() {
     return locator;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/db66e6cc/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncMetaRegionLocator.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncMetaRegionLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncMetaRegionLocator.java
index 06b5b57..9fef15d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncMetaRegionLocator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncMetaRegionLocator.java
@@ -17,11 +17,16 @@
  */
 package org.apache.hadoop.hbase.client;
 
-import static org.apache.hadoop.hbase.client.AsyncRegionLocator.*;
+import static org.apache.hadoop.hbase.client.AsyncRegionLocatorHelper.canUpdateOnError;
+import static org.apache.hadoop.hbase.client.AsyncRegionLocatorHelper.createRegionLocations;
+import static org.apache.hadoop.hbase.client.AsyncRegionLocatorHelper.isGood;
+import static org.apache.hadoop.hbase.client.AsyncRegionLocatorHelper.removeRegionLocation;
+import static org.apache.hadoop.hbase.client.AsyncRegionLocatorHelper.replaceRegionLocation;
+
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.atomic.AtomicReference;
-
 import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.RegionLocations;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -36,43 +41,43 @@ class AsyncMetaRegionLocator {
 
   private final AsyncRegistry registry;
 
-  private final AtomicReference<HRegionLocation> metaRegionLocation = new AtomicReference<>();
+  private final AtomicReference<RegionLocations> metaRegionLocations = new AtomicReference<>();
 
-  private final AtomicReference<CompletableFuture<HRegionLocation>> metaRelocateFuture =
-      new AtomicReference<>();
+  private final AtomicReference<CompletableFuture<RegionLocations>> metaRelocateFuture =
+    new AtomicReference<>();
 
   AsyncMetaRegionLocator(AsyncRegistry registry) {
     this.registry = registry;
   }
 
-  CompletableFuture<HRegionLocation> getRegionLocation(boolean reload) {
+  /**
+   * Get the region locations for meta region. If the location for the given replica is not
+   * available in the cached locations, then fetch from the HBase cluster.
+   * <p/>
+   * The <code>replicaId</code> parameter is important. If the region replication config for meta
+   * region is changed, then the cached region locations may not have the locations for new
+   * replicas. If we do not check the location for the given replica, we will always return the
+   * cached region locations and cause an infinite loop.
+   */
+  CompletableFuture<RegionLocations> getRegionLocations(int replicaId, boolean reload) {
     for (;;) {
       if (!reload) {
-        HRegionLocation metaRegionLocation = this.metaRegionLocation.get();
-        if (metaRegionLocation != null) {
-          return CompletableFuture.completedFuture(metaRegionLocation);
+        RegionLocations locs = this.metaRegionLocations.get();
+        if (isGood(locs, replicaId)) {
+          return CompletableFuture.completedFuture(locs);
         }
       }
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("Meta region location cache is null, try fetching from registry.");
-      }
+      LOG.trace("Meta region location cache is null, try fetching from registry.");
       if (metaRelocateFuture.compareAndSet(null, new CompletableFuture<>())) {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Start fetching meta region location from registry.");
-        }
-        CompletableFuture<HRegionLocation> future = metaRelocateFuture.get();
+        LOG.debug("Start fetching meta region location from registry.");
+        CompletableFuture<RegionLocations> future = metaRelocateFuture.get();
         registry.getMetaRegionLocation().whenComplete((locs, error) -> {
           if (error != null) {
-            if (LOG.isDebugEnabled()) {
-              LOG.debug("Failed to fetch meta region location from registry", error);
-            }
+            LOG.debug("Failed to fetch meta region location from registry", error);
             metaRelocateFuture.getAndSet(null).completeExceptionally(error);
             return;
           }
-          HRegionLocation loc = locs.getDefaultRegionLocation();
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("The fetched meta region location is " + loc);
-          }
+          LOG.debug("The fetched meta region location is {}" + locs);
           // Here we update cache before reset future, so it is possible that someone can get a
           // stale value. Consider this:
           // 1. update cache
@@ -82,12 +87,12 @@ class AsyncMetaRegionLocator {
           // cleared in step 2.
           // But we do not think it is a big deal as it rarely happens, and even if it happens, the
           // caller will retry again later, no correctness problems.
-          this.metaRegionLocation.set(loc);
+          this.metaRegionLocations.set(locs);
           metaRelocateFuture.set(null);
-          future.complete(loc);
+          future.complete(locs);
         });
       } else {
-        CompletableFuture<HRegionLocation> future = metaRelocateFuture.get();
+        CompletableFuture<RegionLocations> future = metaRelocateFuture.get();
         if (future != null) {
           return future;
         }
@@ -95,30 +100,56 @@ class AsyncMetaRegionLocator {
     }
   }
 
-  void updateCachedLocation(HRegionLocation loc, Throwable exception) {
-    AsyncRegionLocator.updateCachedLocation(loc, exception, l -> metaRegionLocation.get(),
-      newLoc -> {
-        for (;;) {
-          HRegionLocation oldLoc = metaRegionLocation.get();
-          if (oldLoc != null && (oldLoc.getSeqNum() > newLoc.getSeqNum() ||
-              oldLoc.getServerName().equals(newLoc.getServerName()))) {
-            return;
-          }
-          if (metaRegionLocation.compareAndSet(oldLoc, newLoc)) {
-            return;
-          }
-        }
-      }, l -> {
-        for (;;) {
-          HRegionLocation oldLoc = metaRegionLocation.get();
-          if (!canUpdate(l, oldLoc) || metaRegionLocation.compareAndSet(oldLoc, null)) {
-            return;
-          }
+  private HRegionLocation getCacheLocation(HRegionLocation loc) {
+    RegionLocations locs = metaRegionLocations.get();
+    return locs != null ? locs.getRegionLocation(loc.getRegion().getReplicaId()) : null;
+  }
+
+  private void addLocationToCache(HRegionLocation loc) {
+    for (;;) {
+      int replicaId = loc.getRegion().getReplicaId();
+      RegionLocations oldLocs = metaRegionLocations.get();
+      if (oldLocs == null) {
+        RegionLocations newLocs = createRegionLocations(loc);
+        if (metaRegionLocations.compareAndSet(null, newLocs)) {
+          return;
         }
-      });
+      }
+      HRegionLocation oldLoc = oldLocs.getRegionLocation(replicaId);
+      if (oldLoc != null && (oldLoc.getSeqNum() > loc.getSeqNum() ||
+        oldLoc.getServerName().equals(loc.getServerName()))) {
+        return;
+      }
+      RegionLocations newLocs = replaceRegionLocation(oldLocs, loc);
+      if (metaRegionLocations.compareAndSet(oldLocs, newLocs)) {
+        return;
+      }
+    }
+  }
+
+  private void removeLocationFromCache(HRegionLocation loc) {
+    for (;;) {
+      RegionLocations oldLocs = metaRegionLocations.get();
+      if (oldLocs == null) {
+        return;
+      }
+      HRegionLocation oldLoc = oldLocs.getRegionLocation(loc.getRegion().getReplicaId());
+      if (!canUpdateOnError(loc, oldLoc)) {
+        return;
+      }
+      RegionLocations newLocs = removeRegionLocation(oldLocs, loc.getRegion().getReplicaId());
+      if (metaRegionLocations.compareAndSet(oldLocs, newLocs)) {
+        return;
+      }
+    }
+  }
+
+  void updateCachedLocationOnError(HRegionLocation loc, Throwable exception) {
+    AsyncRegionLocatorHelper.updateCachedLocationOnError(loc, exception, this::getCacheLocation,
+      this::addLocationToCache, this::removeLocationFromCache);
   }
 
   void clearCache() {
-    metaRegionLocation.set(null);
+    metaRegionLocations.set(null);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/db66e6cc/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java
index 7e3d56c..1fcfbb0 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java
@@ -20,6 +20,11 @@ package org.apache.hadoop.hbase.client;
 import static org.apache.hadoop.hbase.HConstants.NINES;
 import static org.apache.hadoop.hbase.HConstants.ZEROES;
 import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME;
+import static org.apache.hadoop.hbase.client.AsyncRegionLocatorHelper.canUpdateOnError;
+import static org.apache.hadoop.hbase.client.AsyncRegionLocatorHelper.createRegionLocations;
+import static org.apache.hadoop.hbase.client.AsyncRegionLocatorHelper.isGood;
+import static org.apache.hadoop.hbase.client.AsyncRegionLocatorHelper.mergeRegionLocations;
+import static org.apache.hadoop.hbase.client.AsyncRegionLocatorHelper.removeRegionLocation;
 import static org.apache.hadoop.hbase.client.ConnectionUtils.createClosestRowAfter;
 import static org.apache.hadoop.hbase.client.ConnectionUtils.isEmptyStopRow;
 import static org.apache.hadoop.hbase.client.RegionInfo.createRegionName;
@@ -39,7 +44,9 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.ConcurrentNavigableMap;
 import java.util.concurrent.ConcurrentSkipListMap;
+import org.apache.commons.lang3.ObjectUtils;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.HBaseIOException;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.MetaTableAccessor;
@@ -53,6 +60,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.hbase.thirdparty.com.google.common.base.Objects;
 
 /**
  * The asynchronous locator for regions other than meta.
@@ -83,9 +91,9 @@ class AsyncNonMetaRegionLocator {
 
   private static final class LocateRequest {
 
-    public final byte[] row;
+    private final byte[] row;
 
-    public final RegionLocateType locateType;
+    private final RegionLocateType locateType;
 
     public LocateRequest(byte[] row, RegionLocateType locateType) {
       this.row = row;
@@ -109,12 +117,12 @@ class AsyncNonMetaRegionLocator {
 
   private static final class TableCache {
 
-    public final ConcurrentNavigableMap<byte[], HRegionLocation> cache =
+    private final ConcurrentNavigableMap<byte[], RegionLocations> cache =
       new ConcurrentSkipListMap<>(BYTES_COMPARATOR);
 
-    public final Set<LocateRequest> pendingRequests = new HashSet<>();
+    private final Set<LocateRequest> pendingRequests = new HashSet<>();
 
-    public final Map<LocateRequest, CompletableFuture<HRegionLocation>> allRequests =
+    private final Map<LocateRequest, CompletableFuture<RegionLocations>> allRequests =
       new LinkedHashMap<>();
 
     public boolean hasQuota(int max) {
@@ -133,25 +141,29 @@ class AsyncNonMetaRegionLocator {
       return allRequests.keySet().stream().filter(r -> !isPending(r)).findFirst();
     }
 
-    public void clearCompletedRequests(Optional<HRegionLocation> location) {
-      for (Iterator<Map.Entry<LocateRequest, CompletableFuture<HRegionLocation>>> iter =
+    public void clearCompletedRequests(Optional<RegionLocations> locations) {
+      for (Iterator<Map.Entry<LocateRequest, CompletableFuture<RegionLocations>>> iter =
         allRequests.entrySet().iterator(); iter.hasNext();) {
-        Map.Entry<LocateRequest, CompletableFuture<HRegionLocation>> entry = iter.next();
-        if (tryComplete(entry.getKey(), entry.getValue(), location)) {
+        Map.Entry<LocateRequest, CompletableFuture<RegionLocations>> entry = iter.next();
+        if (tryComplete(entry.getKey(), entry.getValue(), locations)) {
           iter.remove();
         }
       }
     }
 
-    private boolean tryComplete(LocateRequest req, CompletableFuture<HRegionLocation> future,
-        Optional<HRegionLocation> location) {
+    private boolean tryComplete(LocateRequest req, CompletableFuture<RegionLocations> future,
+        Optional<RegionLocations> locations) {
       if (future.isDone()) {
         return true;
       }
-      if (!location.isPresent()) {
+      if (!locations.isPresent()) {
         return false;
       }
-      HRegionLocation loc = location.get();
+      RegionLocations locs = locations.get();
+      HRegionLocation loc = ObjectUtils.firstNonNull(locs.getRegionLocations());
+      // we should at least have one location available, otherwise the request should fail and
+      // should not arrive here
+      assert loc != null;
       boolean completed;
       if (req.locateType.equals(RegionLocateType.BEFORE)) {
         // for locating the row before current row, the common case is to find the previous region
@@ -166,7 +178,7 @@ class AsyncNonMetaRegionLocator {
         completed = loc.getRegion().containsRow(req.row);
       }
       if (completed) {
-        future.complete(loc);
+        future.complete(locs);
         return true;
       } else {
         return false;
@@ -186,59 +198,59 @@ class AsyncNonMetaRegionLocator {
     return computeIfAbsent(cache, tableName, TableCache::new);
   }
 
-  private void removeFromCache(HRegionLocation loc) {
-    TableCache tableCache = cache.get(loc.getRegion().getTable());
-    if (tableCache == null) {
-      return;
+  private boolean isEqual(RegionLocations locs1, RegionLocations locs2) {
+    HRegionLocation[] locArr1 = locs1.getRegionLocations();
+    HRegionLocation[] locArr2 = locs2.getRegionLocations();
+    if (locArr1.length != locArr2.length) {
+      return false;
     }
-    tableCache.cache.computeIfPresent(loc.getRegion().getStartKey(), (k, oldLoc) -> {
-      if (oldLoc.getSeqNum() > loc.getSeqNum() ||
-        !oldLoc.getServerName().equals(loc.getServerName())) {
-        return oldLoc;
+    for (int i = 0; i < locArr1.length; i++) {
+      // do not need to compare region info
+      HRegionLocation loc1 = locArr1[i];
+      HRegionLocation loc2 = locArr2[i];
+      if (loc1 == null) {
+        if (loc2 != null) {
+          return false;
+        }
+      } else {
+        if (loc2 == null) {
+          return false;
+        }
+        if (loc1.getSeqNum() != loc2.getSeqNum()) {
+          return false;
+        }
+        if (Objects.equal(loc1.getServerName(), loc2.getServerName())) {
+          return false;
+        }
       }
-      return null;
-    });
+    }
+    return true;
   }
 
   // return whether we add this loc to cache
-  private boolean addToCache(TableCache tableCache, HRegionLocation loc) {
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("Try adding " + loc + " to cache");
-    }
-    byte[] startKey = loc.getRegion().getStartKey();
-    HRegionLocation oldLoc = tableCache.cache.putIfAbsent(startKey, loc);
-    if (oldLoc == null) {
-      return true;
-    }
-    if (oldLoc.getSeqNum() > loc.getSeqNum() ||
-      oldLoc.getServerName().equals(loc.getServerName())) {
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("Will not add " + loc + " to cache because the old value " + oldLoc +
-          " is newer than us or has the same server name");
-      }
-      return false;
-    }
-    return loc == tableCache.cache.compute(startKey, (k, oldValue) -> {
-      if (oldValue == null || oldValue.getSeqNum() <= loc.getSeqNum()) {
-        return loc;
+  private boolean addToCache(TableCache tableCache, RegionLocations locs) {
+    LOG.trace("Try adding {} to cache", locs);
+    byte[] startKey = locs.getDefaultRegionLocation().getRegion().getStartKey();
+    for (;;) {
+      RegionLocations oldLocs = tableCache.cache.putIfAbsent(startKey, locs);
+      if (oldLocs == null) {
+        return true;
       }
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("Will not add " + loc + " to cache because the old value " + oldValue +
+      RegionLocations mergedLocs = mergeRegionLocations(locs, oldLocs);
+      if (isEqual(mergedLocs, oldLocs)) {
+        // the merged one is the same with the old one, give up
+        LOG.trace("Will not add {} to cache because the old value {} " +
           " is newer than us or has the same server name." +
-          " Maybe it is updated before we replace it");
+          " Maybe it is updated before we replace it", locs, oldLocs);
+        return false;
       }
-      return oldValue;
-    });
-  }
-
-  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "UPM_UNCALLED_PRIVATE_METHOD",
-      justification = "Called by lambda expression")
-  private void addToCache(HRegionLocation loc) {
-    addToCache(getTableCache(loc.getRegion().getTable()), loc);
-    LOG.trace("Try adding {} to cache", loc);
+      if (tableCache.cache.replace(startKey, oldLocs, mergedLocs)) {
+        return true;
+      }
+    }
   }
 
-  private void complete(TableName tableName, LocateRequest req, HRegionLocation loc,
+  private void complete(TableName tableName, LocateRequest req, RegionLocations locs,
       Throwable error) {
     if (error != null) {
       LOG.warn("Failed to locate region in '" + tableName + "', row='" +
@@ -246,8 +258,8 @@ class AsyncNonMetaRegionLocator {
     }
     Optional<LocateRequest> toSend = Optional.empty();
     TableCache tableCache = getTableCache(tableName);
-    if (loc != null) {
-      if (!addToCache(tableCache, loc)) {
+    if (locs != null) {
+      if (!addToCache(tableCache, locs)) {
         // someone is ahead of us.
         synchronized (tableCache) {
           tableCache.pendingRequests.remove(req);
@@ -269,7 +281,7 @@ class AsyncNonMetaRegionLocator {
           future.completeExceptionally(error);
         }
       }
-      tableCache.clearCompletedRequests(Optional.ofNullable(loc));
+      tableCache.clearCompletedRequests(Optional.ofNullable(locs));
       // Remove a complete locate request in a synchronized block, so the table cache must have
       // quota to send a candidate request.
       toSend = tableCache.getCandidate();
@@ -286,9 +298,11 @@ class AsyncNonMetaRegionLocator {
         Bytes.toStringBinary(req.row), req.locateType, locs);
     }
 
+    // the default region location should always be presented when fetching from meta, otherwise
+    // let's fail the request.
     if (locs == null || locs.getDefaultRegionLocation() == null) {
       complete(tableName, req, null,
-        new IOException(String.format("No location found for '%s', row='%s', locateType=%s",
+        new HBaseIOException(String.format("No location found for '%s', row='%s', locateType=%s",
           tableName, Bytes.toStringBinary(req.row), req.locateType)));
       return true;
     }
@@ -296,58 +310,60 @@ class AsyncNonMetaRegionLocator {
     RegionInfo info = loc.getRegion();
     if (info == null) {
       complete(tableName, req, null,
-        new IOException(String.format("HRegionInfo is null for '%s', row='%s', locateType=%s",
+        new HBaseIOException(String.format("HRegionInfo is null for '%s', row='%s', locateType=%s",
           tableName, Bytes.toStringBinary(req.row), req.locateType)));
       return true;
     }
     if (info.isSplitParent()) {
       return false;
     }
-    if (loc.getServerName() == null) {
-      complete(tableName, req, null,
-        new IOException(
-          String.format("No server address listed for region '%s', row='%s', locateType=%s",
-            info.getRegionNameAsString(), Bytes.toStringBinary(req.row), req.locateType)));
-      return true;
-    }
-    complete(tableName, req, loc, null);
+    complete(tableName, req, locs, null);
     return true;
   }
 
-  private HRegionLocation locateRowInCache(TableCache tableCache, TableName tableName, byte[] row) {
-    Map.Entry<byte[], HRegionLocation> entry = tableCache.cache.floorEntry(row);
+  private RegionLocations locateRowInCache(TableCache tableCache, TableName tableName, byte[] row,
+      int replicaId) {
+    Map.Entry<byte[], RegionLocations> entry = tableCache.cache.floorEntry(row);
     if (entry == null) {
       return null;
     }
-    HRegionLocation loc = entry.getValue();
+    RegionLocations locs = entry.getValue();
+    HRegionLocation loc = locs.getRegionLocation(replicaId);
+    if (loc == null) {
+      return null;
+    }
     byte[] endKey = loc.getRegion().getEndKey();
     if (isEmptyStopRow(endKey) || Bytes.compareTo(row, endKey) < 0) {
       if (LOG.isTraceEnabled()) {
-        LOG.trace("Found " + loc + " in cache for '" + tableName + "', row='" +
-          Bytes.toStringBinary(row) + "', locateType=" + RegionLocateType.CURRENT);
+        LOG.trace("Found {} in cache for {}, row='{}', locateType={}, replicaId={}", loc, tableName,
+          Bytes.toStringBinary(row), RegionLocateType.CURRENT, replicaId);
       }
-      return loc;
+      return locs;
     } else {
       return null;
     }
   }
 
-  private HRegionLocation locateRowBeforeInCache(TableCache tableCache, TableName tableName,
-      byte[] row) {
+  private RegionLocations locateRowBeforeInCache(TableCache tableCache, TableName tableName,
+      byte[] row, int replicaId) {
     boolean isEmptyStopRow = isEmptyStopRow(row);
-    Map.Entry<byte[], HRegionLocation> entry =
-        isEmptyStopRow ? tableCache.cache.lastEntry() : tableCache.cache.lowerEntry(row);
+    Map.Entry<byte[], RegionLocations> entry =
+      isEmptyStopRow ? tableCache.cache.lastEntry() : tableCache.cache.lowerEntry(row);
     if (entry == null) {
       return null;
     }
-    HRegionLocation loc = entry.getValue();
+    RegionLocations locs = entry.getValue();
+    HRegionLocation loc = locs.getRegionLocation(replicaId);
+    if (loc == null) {
+      return null;
+    }
     if (isEmptyStopRow(loc.getRegion().getEndKey()) ||
       (!isEmptyStopRow && Bytes.compareTo(loc.getRegion().getEndKey(), row) >= 0)) {
       if (LOG.isTraceEnabled()) {
-        LOG.trace("Found " + loc + " in cache for '" + tableName + "', row='" +
-          Bytes.toStringBinary(row) + "', locateType=" + RegionLocateType.BEFORE);
+        LOG.trace("Found {} in cache for {}, row='{}', locateType={}, replicaId={}", loc, tableName,
+          Bytes.toStringBinary(row), RegionLocateType.BEFORE, replicaId);
       }
-      return loc;
+      return locs;
     } else {
       return null;
     }
@@ -390,8 +406,8 @@ class AsyncNonMetaRegionLocator {
             if (tableNotFound) {
               complete(tableName, req, null, new TableNotFoundException(tableName));
             } else if (!completeNormally) {
-              complete(tableName, req, null, new IOException(
-                "Unable to find region for " + Bytes.toStringBinary(req.row) + " in " + tableName));
+              complete(tableName, req, null, new IOException("Unable to find region for '" +
+                Bytes.toStringBinary(req.row) + "' in " + tableName));
             }
           }
 
@@ -423,13 +439,12 @@ class AsyncNonMetaRegionLocator {
                   continue;
                 }
                 RegionInfo info = loc.getRegion();
-                if (info == null || info.isOffline() || info.isSplitParent() ||
-                  loc.getServerName() == null) {
+                if (info == null || info.isOffline() || info.isSplitParent()) {
                   continue;
                 }
-                if (addToCache(tableCache, loc)) {
+                if (addToCache(tableCache, locs)) {
                   synchronized (tableCache) {
-                    tableCache.clearCompletedRequests(Optional.of(loc));
+                    tableCache.clearCompletedRequests(Optional.of(locs));
                   }
                 }
               }
@@ -438,36 +453,36 @@ class AsyncNonMetaRegionLocator {
         });
   }
 
-  private HRegionLocation locateInCache(TableCache tableCache, TableName tableName, byte[] row,
-      RegionLocateType locateType) {
+  private RegionLocations locateInCache(TableCache tableCache, TableName tableName, byte[] row,
+      int replicaId, RegionLocateType locateType) {
     return locateType.equals(RegionLocateType.BEFORE)
-      ? locateRowBeforeInCache(tableCache, tableName, row)
-      : locateRowInCache(tableCache, tableName, row);
+      ? locateRowBeforeInCache(tableCache, tableName, row, replicaId)
+      : locateRowInCache(tableCache, tableName, row, replicaId);
   }
 
   // locateToPrevious is true means we will use the start key of a region to locate the region
   // placed before it. Used for reverse scan. See the comment of
   // AsyncRegionLocator.getPreviousRegionLocation.
-  private CompletableFuture<HRegionLocation> getRegionLocationInternal(TableName tableName,
-      byte[] row, RegionLocateType locateType, boolean reload) {
+  private CompletableFuture<RegionLocations> getRegionLocationsInternal(TableName tableName,
+      byte[] row, int replicaId, RegionLocateType locateType, boolean reload) {
     // AFTER should be convert to CURRENT before calling this method
     assert !locateType.equals(RegionLocateType.AFTER);
     TableCache tableCache = getTableCache(tableName);
     if (!reload) {
-      HRegionLocation loc = locateInCache(tableCache, tableName, row, locateType);
-      if (loc != null) {
-        return CompletableFuture.completedFuture(loc);
+      RegionLocations locs = locateInCache(tableCache, tableName, row, replicaId, locateType);
+      if (isGood(locs, replicaId)) {
+        return CompletableFuture.completedFuture(locs);
       }
     }
-    CompletableFuture<HRegionLocation> future;
+    CompletableFuture<RegionLocations> future;
     LocateRequest req;
     boolean sendRequest = false;
     synchronized (tableCache) {
       // check again
       if (!reload) {
-        HRegionLocation loc = locateInCache(tableCache, tableName, row, locateType);
-        if (loc != null) {
-          return CompletableFuture.completedFuture(loc);
+        RegionLocations locs = locateInCache(tableCache, tableName, row, replicaId, locateType);
+        if (isGood(locs, replicaId)) {
+          return CompletableFuture.completedFuture(locs);
         }
       }
       req = new LocateRequest(row, locateType);
@@ -487,28 +502,58 @@ class AsyncNonMetaRegionLocator {
     return future;
   }
 
-  CompletableFuture<HRegionLocation> getRegionLocation(TableName tableName, byte[] row,
-      RegionLocateType locateType, boolean reload) {
-    if (locateType.equals(RegionLocateType.BEFORE)) {
-      return getRegionLocationInternal(tableName, row, locateType, reload);
-    } else {
-      // as we know the exact row after us, so we can just create the new row, and use the same
-      // algorithm to locate it.
-      if (locateType.equals(RegionLocateType.AFTER)) {
-        row = createClosestRowAfter(row);
-      }
-      return getRegionLocationInternal(tableName, row, RegionLocateType.CURRENT, reload);
+  CompletableFuture<RegionLocations> getRegionLocations(TableName tableName, byte[] row,
+      int replicaId, RegionLocateType locateType, boolean reload) {
+    // as we know the exact row after us, so we can just create the new row, and use the same
+    // algorithm to locate it.
+    if (locateType.equals(RegionLocateType.AFTER)) {
+      row = createClosestRowAfter(row);
+      locateType = RegionLocateType.CURRENT;
     }
+    return getRegionLocationsInternal(tableName, row, replicaId, locateType, reload);
   }
 
-  void updateCachedLocation(HRegionLocation loc, Throwable exception) {
-    AsyncRegionLocator.updateCachedLocation(loc, exception, l -> {
-      TableCache tableCache = cache.get(l.getRegion().getTable());
-      if (tableCache == null) {
-        return null;
+  private void removeLocationFromCache(HRegionLocation loc) {
+    TableCache tableCache = cache.get(loc.getRegion().getTable());
+    if (tableCache == null) {
+      return;
+    }
+    byte[] startKey = loc.getRegion().getStartKey();
+    for (;;) {
+      RegionLocations oldLocs = tableCache.cache.get(startKey);
+      HRegionLocation oldLoc = oldLocs.getRegionLocation(loc.getRegion().getReplicaId());
+      if (!canUpdateOnError(loc, oldLoc)) {
+        return;
       }
-      return tableCache.cache.get(l.getRegion().getStartKey());
-    }, this::addToCache, this::removeFromCache);
+      RegionLocations newLocs = removeRegionLocation(oldLocs, loc.getRegion().getReplicaId());
+      if (newLocs == null) {
+        if (tableCache.cache.remove(startKey, oldLocs)) {
+          return;
+        }
+      } else {
+        if (tableCache.cache.replace(startKey, oldLocs, newLocs)) {
+          return;
+        }
+      }
+    }
+  }
+
+  private void addLocationToCache(HRegionLocation loc) {
+    addToCache(getTableCache(loc.getRegion().getTable()), createRegionLocations(loc));
+  }
+
+  private HRegionLocation getCachedLocation(HRegionLocation loc) {
+    TableCache tableCache = cache.get(loc.getRegion().getTable());
+    if (tableCache == null) {
+      return null;
+    }
+    RegionLocations locs = tableCache.cache.get(loc.getRegion().getStartKey());
+    return locs != null ? locs.getRegionLocation(loc.getRegion().getReplicaId()) : null;
+  }
+
+  void updateCachedLocationOnError(HRegionLocation loc, Throwable exception) {
+    AsyncRegionLocatorHelper.updateCachedLocationOnError(loc, exception, this::getCachedLocation,
+      this::addLocationToCache, this::removeLocationFromCache);
   }
 
   void clearCache(TableName tableName) {
@@ -526,11 +571,11 @@ class AsyncNonMetaRegionLocator {
 
   // only used for testing whether we have cached the location for a region.
   @VisibleForTesting
-  HRegionLocation getRegionLocationInCache(TableName tableName, byte[] row) {
+  RegionLocations getRegionLocationInCache(TableName tableName, byte[] row) {
     TableCache tableCache = cache.get(tableName);
     if (tableCache == null) {
       return null;
     }
-    return locateRowInCache(tableCache, tableName, row);
+    return locateRowInCache(tableCache, tableName, row, RegionReplicaUtil.DEFAULT_REPLICA_ID);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/db66e6cc/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java
index 56228ab..d624974 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java
@@ -18,26 +18,24 @@
 package org.apache.hadoop.hbase.client;
 
 import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME;
-import static org.apache.hadoop.hbase.exceptions.ClientExceptionsUtil.findException;
-import static org.apache.hadoop.hbase.exceptions.ClientExceptionsUtil.isMetaClearingException;
-
-import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer;
-import org.apache.hbase.thirdparty.io.netty.util.Timeout;
+import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
 
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.TimeUnit;
-import java.util.function.Consumer;
-import java.util.function.Function;
 import java.util.function.Supplier;
-
+import org.apache.hadoop.hbase.HBaseIOException;
 import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.RegionException;
+import org.apache.hadoop.hbase.RegionLocations;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hbase.exceptions.RegionMovedException;
-import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
-import org.apache.hadoop.hbase.util.Bytes;
+
+import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer;
+import org.apache.hbase.thirdparty.io.netty.util.Timeout;
 
 /**
  * The asynchronous region locator.
@@ -59,8 +57,8 @@ class AsyncRegionLocator {
     this.retryTimer = retryTimer;
   }
 
-  private CompletableFuture<HRegionLocation> withTimeout(CompletableFuture<HRegionLocation> future,
-      long timeoutNs, Supplier<String> timeoutMsg) {
+  private <T> CompletableFuture<T> withTimeout(CompletableFuture<T> future, long timeoutNs,
+      Supplier<String> timeoutMsg) {
     if (future.isDone() || timeoutNs <= 0) {
       return future;
     }
@@ -78,74 +76,75 @@ class AsyncRegionLocator {
     });
   }
 
-  CompletableFuture<HRegionLocation> getRegionLocation(TableName tableName, byte[] row,
+  private boolean isMeta(TableName tableName) {
+    return TableName.isMetaTableName(tableName);
+  }
+
+  CompletableFuture<RegionLocations> getRegionLocations(TableName tableName, byte[] row,
       RegionLocateType type, boolean reload, long timeoutNs) {
+    CompletableFuture<RegionLocations> future = isMeta(tableName)
+      ? metaRegionLocator.getRegionLocations(RegionReplicaUtil.DEFAULT_REPLICA_ID, reload)
+      : nonMetaRegionLocator.getRegionLocations(tableName, row,
+        RegionReplicaUtil.DEFAULT_REPLICA_ID, type, reload);
+    return withTimeout(future, timeoutNs,
+      () -> "Timeout(" + TimeUnit.NANOSECONDS.toMillis(timeoutNs) +
+        "ms) waiting for region locations for " + tableName + ", row='" +
+        Bytes.toStringBinary(row) + "'");
+  }
+
+  CompletableFuture<HRegionLocation> getRegionLocation(TableName tableName, byte[] row,
+      int replicaId, RegionLocateType type, boolean reload, long timeoutNs) {
     // meta region can not be split right now so we always call the same method.
     // Change it later if the meta table can have more than one regions.
-    CompletableFuture<HRegionLocation> future =
-        tableName.equals(META_TABLE_NAME) ? metaRegionLocator.getRegionLocation(reload)
-            : nonMetaRegionLocator.getRegionLocation(tableName, row, type, reload);
+    CompletableFuture<HRegionLocation> future = new CompletableFuture<>();
+    CompletableFuture<RegionLocations> locsFuture =
+      isMeta(tableName) ? metaRegionLocator.getRegionLocations(replicaId, reload)
+        : nonMetaRegionLocator.getRegionLocations(tableName, row, replicaId, type, reload);
+    addListener(locsFuture, (locs, error) -> {
+      if (error != null) {
+        future.completeExceptionally(error);
+        return;
+      }
+      HRegionLocation loc = locs.getRegionLocation(replicaId);
+      if (loc == null) {
+        future
+          .completeExceptionally(new RegionException("No location for " + tableName + ", row='" +
+            Bytes.toStringBinary(row) + "', locateType=" + type + ", replicaId=" + replicaId));
+      } else if (loc.getServerName() == null) {
+        future.completeExceptionally(new HBaseIOException("No server address listed for region '" +
+          loc.getRegion().getRegionNameAsString() + ", row='" + Bytes.toStringBinary(row) +
+          "', locateType=" + type + ", replicaId=" + replicaId));
+      } else {
+        future.complete(loc);
+      }
+    });
     return withTimeout(future, timeoutNs,
       () -> "Timeout(" + TimeUnit.NANOSECONDS.toMillis(timeoutNs) +
-          "ms) waiting for region location for " + tableName + ", row='" +
-          Bytes.toStringBinary(row) + "'");
+        "ms) waiting for region location for " + tableName + ", row='" + Bytes.toStringBinary(row) +
+        "', replicaId=" + replicaId);
   }
 
   CompletableFuture<HRegionLocation> getRegionLocation(TableName tableName, byte[] row,
-      RegionLocateType type, long timeoutNs) {
-    return getRegionLocation(tableName, row, type, false, timeoutNs);
+      int replicaId, RegionLocateType type, long timeoutNs) {
+    return getRegionLocation(tableName, row, replicaId, type, false, timeoutNs);
   }
 
-  static boolean canUpdate(HRegionLocation loc, HRegionLocation oldLoc) {
-    // Do not need to update if no such location, or the location is newer, or the location is not
-    // same with us
-    return oldLoc != null && oldLoc.getSeqNum() <= loc.getSeqNum() &&
-        oldLoc.getServerName().equals(loc.getServerName());
+  CompletableFuture<HRegionLocation> getRegionLocation(TableName tableName, byte[] row,
+      RegionLocateType type, boolean reload, long timeoutNs) {
+    return getRegionLocation(tableName, row, RegionReplicaUtil.DEFAULT_REPLICA_ID, type, reload,
+      timeoutNs);
   }
 
-  static void updateCachedLocation(HRegionLocation loc, Throwable exception,
-      Function<HRegionLocation, HRegionLocation> cachedLocationSupplier,
-      Consumer<HRegionLocation> addToCache, Consumer<HRegionLocation> removeFromCache) {
-    HRegionLocation oldLoc = cachedLocationSupplier.apply(loc);
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Try updating " + loc + ", the old value is " + oldLoc, exception);
-    }
-    if (!canUpdate(loc, oldLoc)) {
-      return;
-    }
-    Throwable cause = findException(exception);
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("The actual exception when updating " + loc, cause);
-    }
-    if (cause == null || !isMetaClearingException(cause)) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(
-          "Will not update " + loc + " because the exception is null or not the one we care about");
-      }
-      return;
-    }
-    if (cause instanceof RegionMovedException) {
-      RegionMovedException rme = (RegionMovedException) cause;
-      HRegionLocation newLoc =
-          new HRegionLocation(loc.getRegionInfo(), rme.getServerName(), rme.getLocationSeqNum());
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(
-          "Try updating " + loc + " with the new location " + newLoc + " constructed by " + rme);
-      }
-      addToCache.accept(newLoc);
-    } else {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Try removing " + loc + " from cache");
-      }
-      removeFromCache.accept(loc);
-    }
+  CompletableFuture<HRegionLocation> getRegionLocation(TableName tableName, byte[] row,
+      RegionLocateType type, long timeoutNs) {
+    return getRegionLocation(tableName, row, type, false, timeoutNs);
   }
 
-  void updateCachedLocation(HRegionLocation loc, Throwable exception) {
+  void updateCachedLocationOnError(HRegionLocation loc, Throwable exception) {
     if (loc.getRegion().isMetaRegion()) {
-      metaRegionLocator.updateCachedLocation(loc, exception);
+      metaRegionLocator.updateCachedLocationOnError(loc, exception);
     } else {
-      nonMetaRegionLocator.updateCachedLocation(loc, exception);
+      nonMetaRegionLocator.updateCachedLocationOnError(loc, exception);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/db66e6cc/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocatorHelper.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocatorHelper.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocatorHelper.java
new file mode 100644
index 0000000..5c9c091
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocatorHelper.java
@@ -0,0 +1,147 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import static org.apache.hadoop.hbase.exceptions.ClientExceptionsUtil.findException;
+import static org.apache.hadoop.hbase.exceptions.ClientExceptionsUtil.isMetaClearingException;
+
+import java.util.Arrays;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import org.apache.commons.lang3.ObjectUtils;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.RegionLocations;
+import org.apache.hadoop.hbase.exceptions.RegionMovedException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Helper class for asynchronous region locator.
+ */
+@InterfaceAudience.Private
+final class AsyncRegionLocatorHelper {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AsyncRegionLocatorHelper.class);
+
+  private AsyncRegionLocatorHelper() {
+  }
+
+  static boolean canUpdateOnError(HRegionLocation loc, HRegionLocation oldLoc) {
+    // Do not need to update if no such location, or the location is newer, or the location is not
+    // the same with us
+    return oldLoc != null && oldLoc.getSeqNum() <= loc.getSeqNum() &&
+      oldLoc.getServerName().equals(loc.getServerName());
+  }
+
+  static void updateCachedLocationOnError(HRegionLocation loc, Throwable exception,
+      Function<HRegionLocation, HRegionLocation> cachedLocationSupplier,
+      Consumer<HRegionLocation> addToCache, Consumer<HRegionLocation> removeFromCache) {
+    HRegionLocation oldLoc = cachedLocationSupplier.apply(loc);
+    LOG.debug("Try updating {} , the old value is {}", loc, oldLoc, exception);
+    if (!canUpdateOnError(loc, oldLoc)) {
+      return;
+    }
+    Throwable cause = findException(exception);
+    LOG.debug("The actual exception when updating {}", loc, cause);
+    if (cause == null || !isMetaClearingException(cause)) {
+      LOG.debug("Will not update {} because the exception is null or not the one we care about",
+        loc);
+      return;
+    }
+    if (cause instanceof RegionMovedException) {
+      RegionMovedException rme = (RegionMovedException) cause;
+      HRegionLocation newLoc =
+        new HRegionLocation(loc.getRegion(), rme.getServerName(), rme.getLocationSeqNum());
+      LOG.debug("Try updating {} with the new location {} constructed by {}", loc, newLoc, rme);
+      addToCache.accept(newLoc);
+    } else {
+      LOG.debug("Try removing {} from cache", loc);
+      removeFromCache.accept(loc);
+    }
+  }
+
+  static RegionLocations createRegionLocations(HRegionLocation loc) {
+    int replicaId = loc.getRegion().getReplicaId();
+    HRegionLocation[] locs = new HRegionLocation[replicaId + 1];
+    locs[replicaId] = loc;
+    return new RegionLocations(locs);
+  }
+
+  /**
+   * Create a new {@link RegionLocations} based on the given {@code oldLocs}, and replace the
+   * location for the given {@code replicaId} with the given {@code loc}.
+   * <p/>
+   * All the {@link RegionLocations} in async locator related class are immutable because we want to
+   * access them concurrently, so here we need to create a new one, instead of calling
+   * {@link RegionLocations#updateLocation(HRegionLocation, boolean, boolean)}.
+   */
+  static RegionLocations replaceRegionLocation(RegionLocations oldLocs, HRegionLocation loc) {
+    int replicaId = loc.getRegion().getReplicaId();
+    HRegionLocation[] locs = oldLocs.getRegionLocations();
+    locs = Arrays.copyOf(locs, Math.max(replicaId + 1, locs.length));
+    locs[replicaId] = loc;
+    return new RegionLocations(locs);
+  }
+
+  /**
+   * Create a new {@link RegionLocations} based on the given {@code oldLocs}, and remove the
+   * location for the given {@code replicaId}.
+   * <p/>
+   * All the {@link RegionLocations} in async locator related class are immutable because we want to
+   * access them concurrently, so here we need to create a new one, instead of calling
+   * {@link RegionLocations#remove(int)}.
+   */
+  static RegionLocations removeRegionLocation(RegionLocations oldLocs, int replicaId) {
+    HRegionLocation[] locs = oldLocs.getRegionLocations();
+    if (locs.length < replicaId + 1) {
+      // Here we do not modify the oldLocs so it is safe to return it.
+      return oldLocs;
+    }
+    locs = Arrays.copyOf(locs, locs.length);
+    locs[replicaId] = null;
+    if (ObjectUtils.firstNonNull(locs) != null) {
+      return new RegionLocations(locs);
+    } else {
+      // if all the locations are null, just return null
+      return null;
+    }
+  }
+
+  /**
+   * Create a new {@link RegionLocations} which is the merging result for the given two
+   * {@link RegionLocations}.
+   * <p/>
+   * All the {@link RegionLocations} in async locator related class are immutable because we want to
+   * access them concurrently, so here we need to create a new one, instead of calling
+   * {@link RegionLocations#mergeLocations(RegionLocations)} directly.
+   */
+  static RegionLocations mergeRegionLocations(RegionLocations newLocs, RegionLocations oldLocs) {
+    RegionLocations locs = new RegionLocations(newLocs.getRegionLocations());
+    locs.mergeLocations(oldLocs);
+    return locs;
+  }
+
+  static boolean isGood(RegionLocations locs, int replicaId) {
+    if (locs == null) {
+      return false;
+    }
+    HRegionLocation loc = locs.getRegionLocation(replicaId);
+    return loc != null && loc.getServerName() != null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/db66e6cc/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRpcRetryingCaller.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRpcRetryingCaller.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRpcRetryingCaller.java
index d30012f..e03049a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRpcRetryingCaller.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRpcRetryingCaller.java
@@ -88,15 +88,15 @@ public abstract class AsyncRpcRetryingCaller<T> {
     return TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNs);
   }
 
-  protected long remainingTimeNs() {
+  protected final long remainingTimeNs() {
     return operationTimeoutNs - (System.nanoTime() - startNs);
   }
 
-  protected void completeExceptionally() {
+  protected final void completeExceptionally() {
     future.completeExceptionally(new RetriesExhaustedException(tries - 1, exceptions));
   }
 
-  protected void resetCallTimeout() {
+  protected final void resetCallTimeout() {
     long callTimeoutNs;
     if (operationTimeoutNs > 0) {
       callTimeoutNs = remainingTimeNs();
@@ -111,8 +111,15 @@ public abstract class AsyncRpcRetryingCaller<T> {
     resetController(controller, callTimeoutNs);
   }
 
-  protected void onError(Throwable error, Supplier<String> errMsg,
+  protected final void onError(Throwable error, Supplier<String> errMsg,
       Consumer<Throwable> updateCachedLocation) {
+    if (future.isDone()) {
+      // Give up if the future is already done, this is possible if user has already canceled the
+      // future. And for timeline consistent read, we will also cancel some requests if we have
+      // already get one of the responses.
+      LOG.debug("The future is already done, canceled={}, give up retrying", future.isCancelled());
+      return;
+    }
     error = translateException(error);
     if (error instanceof DoNotRetryIOException) {
       future.completeExceptionally(error);

http://git-wip-us.apache.org/repos/asf/hbase/blob/db66e6cc/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRpcRetryingCallerFactory.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRpcRetryingCallerFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRpcRetryingCallerFactory.java
index f80b4e5..a660e74 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRpcRetryingCallerFactory.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRpcRetryingCallerFactory.java
@@ -17,22 +17,22 @@
  */
 package org.apache.hadoop.hbase.client;
 
+import static org.apache.hadoop.hbase.client.ConnectionUtils.retries2Attempts;
 import static org.apache.hbase.thirdparty.com.google.common.base.Preconditions.checkArgument;
 import static org.apache.hbase.thirdparty.com.google.common.base.Preconditions.checkNotNull;
-import static org.apache.hadoop.hbase.client.ConnectionUtils.retries2Attempts;
-
-import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer;
 
 import java.util.List;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.TimeUnit;
-
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
 import org.apache.hadoop.hbase.ipc.HBaseRpcController;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer;
+
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse;
 
@@ -75,6 +75,8 @@ class AsyncRpcRetryingCallerFactory {
 
     private RegionLocateType locateType = RegionLocateType.CURRENT;
 
+    private int replicaId = RegionReplicaUtil.DEFAULT_REPLICA_ID;
+
     public SingleRequestCallerBuilder<T> table(TableName tableName) {
       this.tableName = tableName;
       return this;
@@ -121,11 +123,17 @@ class AsyncRpcRetryingCallerFactory {
       return this;
     }
 
+    public SingleRequestCallerBuilder<T> replicaId(int replicaId) {
+      this.replicaId = replicaId;
+      return this;
+    }
+
     public AsyncSingleRequestRpcRetryingCaller<T> build() {
+      checkArgument(replicaId >= 0, "invalid replica id %s", replicaId);
       return new AsyncSingleRequestRpcRetryingCaller<>(retryTimer, conn,
-          checkNotNull(tableName, "tableName is null"), checkNotNull(row, "row is null"),
-          checkNotNull(locateType, "locateType is null"), checkNotNull(callable, "action is null"),
-          pauseNs, maxAttempts, operationTimeoutNs, rpcTimeoutNs, startLogErrorsCnt);
+        checkNotNull(tableName, "tableName is null"), checkNotNull(row, "row is null"), replicaId,
+        checkNotNull(locateType, "locateType is null"), checkNotNull(callable, "action is null"),
+        pauseNs, maxAttempts, operationTimeoutNs, rpcTimeoutNs, startLogErrorsCnt);
     }
 
     /**
@@ -241,11 +249,11 @@ class AsyncRpcRetryingCallerFactory {
     public AsyncScanSingleRegionRpcRetryingCaller build() {
       checkArgument(scannerId != null, "invalid scannerId %d", scannerId);
       return new AsyncScanSingleRegionRpcRetryingCaller(retryTimer, conn,
-          checkNotNull(scan, "scan is null"), scanMetrics, scannerId,
-          checkNotNull(resultCache, "resultCache is null"),
-          checkNotNull(consumer, "consumer is null"), checkNotNull(stub, "stub is null"),
-          checkNotNull(loc, "location is null"), isRegionServerRemote, scannerLeaseTimeoutPeriodNs,
-          pauseNs, maxAttempts, scanTimeoutNs, rpcTimeoutNs, startLogErrorsCnt);
+        checkNotNull(scan, "scan is null"), scanMetrics, scannerId,
+        checkNotNull(resultCache, "resultCache is null"),
+        checkNotNull(consumer, "consumer is null"), checkNotNull(stub, "stub is null"),
+        checkNotNull(loc, "location is null"), isRegionServerRemote, scannerLeaseTimeoutPeriodNs,
+        pauseNs, maxAttempts, scanTimeoutNs, rpcTimeoutNs, startLogErrorsCnt);
     }
 
     /**
@@ -311,7 +319,7 @@ class AsyncRpcRetryingCallerFactory {
 
     public <T> AsyncBatchRpcRetryingCaller<T> build() {
       return new AsyncBatchRpcRetryingCaller<>(retryTimer, conn, tableName, actions, pauseNs,
-          maxAttempts, operationTimeoutNs, rpcTimeoutNs, startLogErrorsCnt);
+        maxAttempts, operationTimeoutNs, rpcTimeoutNs, startLogErrorsCnt);
     }
 
     public <T> List<CompletableFuture<T>> call() {
@@ -363,8 +371,8 @@ class AsyncRpcRetryingCallerFactory {
 
     public AsyncMasterRequestRpcRetryingCaller<T> build() {
       return new AsyncMasterRequestRpcRetryingCaller<T>(retryTimer, conn,
-          checkNotNull(callable, "action is null"), pauseNs, maxAttempts, operationTimeoutNs,
-          rpcTimeoutNs, startLogErrorsCnt);
+        checkNotNull(callable, "action is null"), pauseNs, maxAttempts, operationTimeoutNs,
+        rpcTimeoutNs, startLogErrorsCnt);
     }
 
     /**
@@ -390,7 +398,8 @@ class AsyncRpcRetryingCallerFactory {
 
     private ServerName serverName;
 
-    public AdminRequestCallerBuilder<T> action(AsyncAdminRequestRetryingCaller.Callable<T> callable) {
+    public AdminRequestCallerBuilder<T> action(
+        AsyncAdminRequestRetryingCaller.Callable<T> callable) {
       this.callable = callable;
       return this;
     }
@@ -420,15 +429,15 @@ class AsyncRpcRetryingCallerFactory {
       return this;
     }
 
-    public AdminRequestCallerBuilder<T> serverName(ServerName serverName){
+    public AdminRequestCallerBuilder<T> serverName(ServerName serverName) {
       this.serverName = serverName;
       return this;
     }
 
     public AsyncAdminRequestRetryingCaller<T> build() {
       return new AsyncAdminRequestRetryingCaller<T>(retryTimer, conn, pauseNs, maxAttempts,
-          operationTimeoutNs, rpcTimeoutNs, startLogErrorsCnt, checkNotNull(serverName,
-            "serverName is null"), checkNotNull(callable, "action is null"));
+        operationTimeoutNs, rpcTimeoutNs, startLogErrorsCnt,
+        checkNotNull(serverName, "serverName is null"), checkNotNull(callable, "action is null"));
     }
 
     public CompletableFuture<T> call() {
@@ -436,7 +445,7 @@ class AsyncRpcRetryingCallerFactory {
     }
   }
 
-  public <T> AdminRequestCallerBuilder<T> adminRequest(){
+  public <T> AdminRequestCallerBuilder<T> adminRequest() {
     return new AdminRequestCallerBuilder<>();
   }
 
@@ -488,8 +497,8 @@ class AsyncRpcRetryingCallerFactory {
 
     public AsyncServerRequestRpcRetryingCaller<T> build() {
       return new AsyncServerRequestRpcRetryingCaller<T>(retryTimer, conn, pauseNs, maxAttempts,
-          operationTimeoutNs, rpcTimeoutNs, startLogErrorsCnt, checkNotNull(serverName,
-            "serverName is null"), checkNotNull(callable, "action is null"));
+        operationTimeoutNs, rpcTimeoutNs, startLogErrorsCnt,
+        checkNotNull(serverName, "serverName is null"), checkNotNull(callable, "action is null"));
     }
 
     public CompletableFuture<T> call() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/db66e6cc/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncSingleRequestRpcRetryingCaller.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncSingleRequestRpcRetryingCaller.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncSingleRequestRpcRetryingCaller.java
index 56c82fb..1a52e5c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncSingleRequestRpcRetryingCaller.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncSingleRequestRpcRetryingCaller.java
@@ -17,17 +17,19 @@
  */
 package org.apache.hadoop.hbase.client;
 
-import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer;
+import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
 
 import java.io.IOException;
 import java.util.concurrent.CompletableFuture;
-
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.ipc.HBaseRpcController;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService;
 
 /**
  * Retry caller for a single request, such as get, put, delete, etc.
@@ -45,18 +47,21 @@ class AsyncSingleRequestRpcRetryingCaller<T> extends AsyncRpcRetryingCaller<T> {
 
   private final byte[] row;
 
+  private final int replicaId;
+
   private final RegionLocateType locateType;
 
   private final Callable<T> callable;
 
   public AsyncSingleRequestRpcRetryingCaller(HashedWheelTimer retryTimer, AsyncConnectionImpl conn,
-      TableName tableName, byte[] row, RegionLocateType locateType, Callable<T> callable,
-      long pauseNs, int maxAttempts, long operationTimeoutNs, long rpcTimeoutNs,
-      int startLogErrorsCnt) {
+      TableName tableName, byte[] row, int replicaId, RegionLocateType locateType,
+      Callable<T> callable, long pauseNs, int maxAttempts, long operationTimeoutNs,
+      long rpcTimeoutNs, int startLogErrorsCnt) {
     super(retryTimer, conn, pauseNs, maxAttempts, operationTimeoutNs, rpcTimeoutNs,
-        startLogErrorsCnt);
+      startLogErrorsCnt);
     this.tableName = tableName;
     this.row = row;
+    this.replicaId = replicaId;
     this.locateType = locateType;
     this.callable = callable;
   }
@@ -67,23 +72,22 @@ class AsyncSingleRequestRpcRetryingCaller<T> extends AsyncRpcRetryingCaller<T> {
       stub = conn.getRegionServerStub(loc.getServerName());
     } catch (IOException e) {
       onError(e,
-        () -> "Get async stub to " + loc.getServerName() + " for '" + Bytes.toStringBinary(row)
-            + "' in " + loc.getRegion().getEncodedName() + " of " + tableName + " failed",
-        err -> conn.getLocator().updateCachedLocation(loc, err));
+        () -> "Get async stub to " + loc.getServerName() + " for '" + Bytes.toStringBinary(row) +
+          "' in " + loc.getRegion().getEncodedName() + " of " + tableName + " failed",
+        err -> conn.getLocator().updateCachedLocationOnError(loc, err));
       return;
     }
     resetCallTimeout();
-    callable.call(controller, loc, stub).whenComplete(
-      (result, error) -> {
-        if (error != null) {
-          onError(error,
-            () -> "Call to " + loc.getServerName() + " for '" + Bytes.toStringBinary(row) + "' in "
-                + loc.getRegion().getEncodedName() + " of " + tableName + " failed",
-            err -> conn.getLocator().updateCachedLocation(loc, err));
-          return;
-        }
-        future.complete(result);
-      });
+    callable.call(controller, loc, stub).whenComplete((result, error) -> {
+      if (error != null) {
+        onError(error,
+          () -> "Call to " + loc.getServerName() + " for '" + Bytes.toStringBinary(row) + "' in " +
+            loc.getRegion().getEncodedName() + " of " + tableName + " failed",
+          err -> conn.getLocator().updateCachedLocationOnError(loc, err));
+        return;
+      }
+      future.complete(result);
+    });
   }
 
   @Override
@@ -98,18 +102,17 @@ class AsyncSingleRequestRpcRetryingCaller<T> extends AsyncRpcRetryingCaller<T> {
     } else {
       locateTimeoutNs = -1L;
     }
-    conn.getLocator()
-        .getRegionLocation(tableName, row, locateType, locateTimeoutNs)
-        .whenComplete(
-          (loc, error) -> {
-            if (error != null) {
-              onError(error, () -> "Locate '" + Bytes.toStringBinary(row) + "' in " + tableName
-                  + " failed", err -> {
-              });
-              return;
-            }
-            call(loc);
-          });
+    addListener(
+      conn.getLocator().getRegionLocation(tableName, row, replicaId, locateType, locateTimeoutNs),
+      (loc, error) -> {
+        if (error != null) {
+          onError(error,
+            () -> "Locate '" + Bytes.toStringBinary(row) + "' in " + tableName + " failed", err -> {
+            });
+          return;
+        }
+        call(loc);
+      });
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/db66e6cc/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocator.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocator.java
index dbfcef5..3bda38e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocator.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.client;
 
 import java.util.concurrent.CompletableFuture;
-
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -55,5 +54,30 @@ public interface AsyncTableRegionLocator {
    * @param row Row to find.
    * @param reload true to reload information or false to use cached information
    */
-  CompletableFuture<HRegionLocation> getRegionLocation(byte[] row, boolean reload);
+  default CompletableFuture<HRegionLocation> getRegionLocation(byte[] row, boolean reload) {
+    return getRegionLocation(row, RegionReplicaUtil.DEFAULT_REPLICA_ID, reload);
+  }
+
+  /**
+   * Finds the region with the given <code>replicaId</code> on which the given row is being served.
+   * <p>
+   * Returns the location of the region with the given <code>replicaId</code> to which the row
+   * belongs.
+   * @param row Row to find.
+   * @param replicaId the replica id of the region
+   */
+  default CompletableFuture<HRegionLocation> getRegionLocation(byte[] row, int replicaId) {
+    return getRegionLocation(row, replicaId, false);
+  }
+
+  /**
+   * Finds the region with the given <code>replicaId</code> on which the given row is being served.
+   * <p>
+   * Returns the location of the region with the given <code>replicaId</code> to which the row
+   * belongs.
+   * @param row Row to find.
+   * @param replicaId the replica id of the region
+   * @param reload true to reload information or false to use cached information
+   */
+  CompletableFuture<HRegionLocation> getRegionLocation(byte[] row, int replicaId, boolean reload);
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/db66e6cc/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocatorImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocatorImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocatorImpl.java
index 7d199df..465a411 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocatorImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocatorImpl.java
@@ -44,7 +44,9 @@ class AsyncTableRegionLocatorImpl implements AsyncTableRegionLocator {
   }
 
   @Override
-  public CompletableFuture<HRegionLocation> getRegionLocation(byte[] row, boolean reload) {
-    return locator.getRegionLocation(tableName, row, RegionLocateType.CURRENT, reload, -1L);
+  public CompletableFuture<HRegionLocation> getRegionLocation(byte[] row, int replicaId,
+      boolean reload) {
+    return locator.getRegionLocation(tableName, row, replicaId, RegionLocateType.CURRENT, reload,
+      -1L);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/db66e6cc/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionConfiguration.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionConfiguration.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionConfiguration.java
index d996004..55c62e7 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionConfiguration.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionConfiguration.java
@@ -38,6 +38,9 @@ public class ConnectionConfiguration {
   public static final long WRITE_BUFFER_PERIODIC_FLUSH_TIMERTICK_MS_DEFAULT = 1000L; // 1 second
   public static final String MAX_KEYVALUE_SIZE_KEY = "hbase.client.keyvalue.maxsize";
   public static final int MAX_KEYVALUE_SIZE_DEFAULT = 10485760;
+  public static final String PRIMARY_CALL_TIMEOUT_MICROSECOND =
+    "hbase.client.primaryCallTimeout.get";
+  public static final int PRIMARY_CALL_TIMEOUT_MICROSECOND_DEFAULT = 10000; // 10ms
 
   private final long writeBufferSize;
   private final long writeBufferPeriodicFlushTimeoutMs;
@@ -86,7 +89,7 @@ public class ConnectionConfiguration {
             HConstants.DEFAULT_HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE);
 
     this.primaryCallTimeoutMicroSecond =
-        conf.getInt("hbase.client.primaryCallTimeout.get", 10000); // 10ms
+      conf.getInt(PRIMARY_CALL_TIMEOUT_MICROSECOND, PRIMARY_CALL_TIMEOUT_MICROSECOND_DEFAULT);
 
     this.replicaCallTimeoutMicroSecondScan =
         conf.getInt("hbase.client.replicaCallTimeout.scan", 1000000); // 1000 ms


[09/15] hbase git commit: HBASE-21662 Add append_peer_exclude_namespaces and remove_peer_exclude_namespaces shell commands

Posted by zh...@apache.org.
HBASE-21662 Add append_peer_exclude_namespaces and remove_peer_exclude_namespaces shell commands

Signed-off-by: Guanghao Zhang <zg...@apache.org>


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

Branch: refs/heads/HBASE-21512
Commit: 466fa920fee572fe20db3b77ebf539dc304d5f31
Parents: db66e6c
Author: meiyi <my...@gamil.com>
Authored: Wed Jan 2 14:08:22 2019 +0800
Committer: Guanghao Zhang <zg...@apache.org>
Committed: Thu Jan 3 10:21:43 2019 +0800

----------------------------------------------------------------------
 .../src/main/ruby/hbase/replication_admin.rb    | 39 +++++++++++
 hbase-shell/src/main/ruby/shell.rb              |  2 +
 .../commands/append_peer_exclude_namespaces.rb  | 47 +++++++++++++
 .../commands/remove_peer_exclude_namespaces.rb  | 45 ++++++++++++
 .../test/ruby/hbase/replication_admin_test.rb   | 74 ++++++++++++++++++++
 5 files changed, 207 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/466fa920/hbase-shell/src/main/ruby/hbase/replication_admin.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/replication_admin.rb b/hbase-shell/src/main/ruby/hbase/replication_admin.rb
index c01b6ea..e061168 100644
--- a/hbase-shell/src/main/ruby/hbase/replication_admin.rb
+++ b/hbase-shell/src/main/ruby/hbase/replication_admin.rb
@@ -285,6 +285,45 @@ module Hbase
       end
     end
 
+    # Append exclude namespaces config for the specified peer
+    def append_peer_exclude_namespaces(id, namespaces)
+      unless namespaces.nil?
+        rpc = get_peer_config(id)
+        unless rpc.nil?
+          if rpc.getExcludeNamespaces.nil?
+            ns_set = java.util.HashSet.new
+          else
+            ns_set = java.util.HashSet.new(rpc.getExcludeNamespaces)
+          end
+          namespaces.each do |n|
+            ns_set.add(n)
+          end
+          builder = ReplicationPeerConfig.newBuilder(rpc)
+          builder.setExcludeNamespaces(ns_set)
+          @admin.updateReplicationPeerConfig(id, builder.build)
+        end
+      end
+    end
+
+    # Remove exclude namespaces config for the specified peer
+    def remove_peer_exclude_namespaces(id, namespaces)
+      unless namespaces.nil?
+        rpc = get_peer_config(id)
+        unless rpc.nil?
+          ns_set = rpc.getExcludeNamespaces
+          unless ns_set.nil?
+            ns_set = java.util.HashSet.new(ns_set)
+            namespaces.each do |n|
+              ns_set.remove(n)
+            end
+          end
+          builder = ReplicationPeerConfig.newBuilder(rpc)
+          builder.setExcludeNamespaces(ns_set)
+          @admin.updateReplicationPeerConfig(id, builder.build)
+        end
+      end
+    end
+
     def set_peer_replicate_all(id, replicate_all)
       rpc = get_peer_config(id)
       return if rpc.nil?

http://git-wip-us.apache.org/repos/asf/hbase/blob/466fa920/hbase-shell/src/main/ruby/shell.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell.rb b/hbase-shell/src/main/ruby/shell.rb
index 1507ca3..1f7eae6 100644
--- a/hbase-shell/src/main/ruby/shell.rb
+++ b/hbase-shell/src/main/ruby/shell.rb
@@ -384,6 +384,8 @@ Shell.load_command_group(
     append_peer_namespaces
     remove_peer_namespaces
     set_peer_exclude_namespaces
+    append_peer_exclude_namespaces
+    remove_peer_exclude_namespaces
     show_peer_tableCFs
     set_peer_tableCFs
     set_peer_exclude_tableCFs

http://git-wip-us.apache.org/repos/asf/hbase/blob/466fa920/hbase-shell/src/main/ruby/shell/commands/append_peer_exclude_namespaces.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/append_peer_exclude_namespaces.rb b/hbase-shell/src/main/ruby/shell/commands/append_peer_exclude_namespaces.rb
new file mode 100644
index 0000000..4f500c8
--- /dev/null
+++ b/hbase-shell/src/main/ruby/shell/commands/append_peer_exclude_namespaces.rb
@@ -0,0 +1,47 @@
+#
+#
+# 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.
+#
+
+module Shell
+  module Commands
+    class AppendPeerExcludeNamespaces < Command
+      def help
+        <<-EOF
+Append the namespaces which not replicated for the specified peer.
+
+Note:
+  1. The replicate_all flag need to be true when append exclude namespaces.
+  2. Append a exclude namespace in the peer config means that all tables in this
+     namespace will not be replicated to the peer cluster. If peer config
+     already has a exclude table, then not allow append this table's namespace
+     as a exclude namespace.
+
+Examples:
+
+    # append ns1,ns2 to be not replicable for peer '2'.
+    hbase> append_peer_exclude_namespaces '2', ["ns1", "ns2"]
+
+        EOF
+      end
+
+      def command(id, namespaces)
+        replication_admin.append_peer_exclude_namespaces(id, namespaces)
+      end
+    end
+  end
+end

http://git-wip-us.apache.org/repos/asf/hbase/blob/466fa920/hbase-shell/src/main/ruby/shell/commands/remove_peer_exclude_namespaces.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/remove_peer_exclude_namespaces.rb b/hbase-shell/src/main/ruby/shell/commands/remove_peer_exclude_namespaces.rb
new file mode 100644
index 0000000..0298ce9
--- /dev/null
+++ b/hbase-shell/src/main/ruby/shell/commands/remove_peer_exclude_namespaces.rb
@@ -0,0 +1,45 @@
+#
+#
+# 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.
+#
+
+module Shell
+  module Commands
+    class RemovePeerExcludeNamespaces < Command
+      def help
+        <<-EOF
+Remove the namespaces which not replicated for the specified peer.
+
+Note:
+  1. The replicate_all flag need to be true when remove exclude namespaces.
+  2. Remove a exclude namespace in the peer config means that all tables in this
+     namespace will be replicated to the peer cluster.
+
+Examples:
+
+    # remove ns1 from the not replicable namespaces for peer '2'.
+    hbase> remove_peer_exclude_namespaces '2', ["ns1", "ns2"]
+
+        EOF
+      end
+
+      def command(id, namespaces)
+        replication_admin.remove_peer_exclude_namespaces(id, namespaces)
+      end
+    end
+  end
+end

http://git-wip-us.apache.org/repos/asf/hbase/blob/466fa920/hbase-shell/src/test/ruby/hbase/replication_admin_test.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/ruby/hbase/replication_admin_test.rb b/hbase-shell/src/test/ruby/hbase/replication_admin_test.rb
index f4c771e..85b4537 100644
--- a/hbase-shell/src/test/ruby/hbase/replication_admin_test.rb
+++ b/hbase-shell/src/test/ruby/hbase/replication_admin_test.rb
@@ -641,6 +641,80 @@ module Hbase
       assert_equal("value2", Bytes.to_string(peer_config.get_peer_data.get(Bytes.toBytes("data2"))))
     end
 
+    define_test "append_peer_exclude_namespaces: works with namespaces array" do
+      cluster_key = "zk4,zk5,zk6:11000:/hbase-test"
+      args = {CLUSTER_KEY => cluster_key}
+      command(:add_peer, @peer_id, args)
+      command(:set_peer_replicate_all, @peer_id, true)
+
+      namespaces = ["ns1", "ns2"]
+      namespaces_str = "!ns1;ns2"
+      command(:append_peer_exclude_namespaces, @peer_id, namespaces)
+      assert_equal(1, command(:list_peers).length)
+      assert_equal(@peer_id, command(:list_peers).get(0).getPeerId)
+      peer_config = command(:list_peers).get(0).getPeerConfig
+      assert_equal(namespaces_str,
+                   replication_admin.show_peer_exclude_namespaces(peer_config))
+
+      namespaces = ["ns3"]
+      namespaces_str = "!ns1;ns2;ns3"
+      command(:append_peer_exclude_namespaces, @peer_id, namespaces)
+      assert_equal(1, command(:list_peers).length)
+      assert_equal(@peer_id, command(:list_peers).get(0).getPeerId)
+      peer_config = command(:list_peers).get(0).getPeerConfig
+      assert_equal(namespaces_str,
+                   replication_admin.show_peer_exclude_namespaces(peer_config))
+
+      # append a namespace which is already excluded in the peer config
+      command(:append_peer_exclude_namespaces, @peer_id, namespaces)
+      assert_equal(1, command(:list_peers).length)
+      assert_equal(@peer_id, command(:list_peers).get(0).getPeerId)
+      peer_config = command(:list_peers).get(0).getPeerConfig
+      assert_equal(namespaces_str,
+                   replication_admin.show_peer_exclude_namespaces(peer_config))
+
+      # cleanup for future tests
+      command(:remove_peer, @peer_id)
+    end
+
+    define_test "remove_peer_exclude_namespaces: works with namespaces array" do
+      cluster_key = "zk4,zk5,zk6:11000:/hbase-test"
+      args = {CLUSTER_KEY => cluster_key}
+      command(:add_peer, @peer_id, args)
+
+      namespaces = ["ns1", "ns2", "ns3"]
+      command(:set_peer_exclude_namespaces, @peer_id, namespaces)
+
+      namespaces = ["ns1", "ns2"]
+      namespaces_str = "!ns3"
+      command(:remove_peer_exclude_namespaces, @peer_id, namespaces)
+      assert_equal(1, command(:list_peers).length)
+      assert_equal(@peer_id, command(:list_peers).get(0).getPeerId)
+      peer_config = command(:list_peers).get(0).getPeerConfig
+      assert_equal(namespaces_str,
+                   replication_admin.show_peer_exclude_namespaces(peer_config))
+
+      namespaces = ["ns3"]
+      namespaces_str = nil
+      command(:remove_peer_exclude_namespaces, @peer_id, namespaces)
+      assert_equal(1, command(:list_peers).length)
+      assert_equal(@peer_id, command(:list_peers).get(0).getPeerId)
+      peer_config = command(:list_peers).get(0).getPeerConfig
+      assert_equal(namespaces_str,
+                   replication_admin.show_peer_exclude_namespaces(peer_config))
+
+      # remove a namespace which is not in peer config
+      command(:remove_peer_namespaces, @peer_id, namespaces)
+      assert_equal(1, command(:list_peers).length)
+      assert_equal(@peer_id, command(:list_peers).get(0).getPeerId)
+      peer_config = command(:list_peers).get(0).getPeerConfig
+      assert_equal(namespaces_str,
+                   replication_admin.show_peer_exclude_namespaces(peer_config))
+
+      # cleanup for future tests
+      command(:remove_peer, @peer_id)
+    end
+
     # assert_raise fails on native exceptions - https://jira.codehaus.org/browse/JRUBY-5279
     # Can't catch native Java exception with assert_raise in JRuby 1.6.8 as in the test below.
     # define_test "add_peer: adding a second peer with same id should error" do


[10/15] hbase git commit: HBASE-21670 Add 2.0.4 to download page

Posted by zh...@apache.org.
HBASE-21670 Add 2.0.4 to download page


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

Branch: refs/heads/HBASE-21512
Commit: c300565311edfa056ddd5abd5480bc7f93fe07a2
Parents: 466fa92
Author: stack <st...@apache.org>
Authored: Thu Jan 3 21:35:14 2019 -0800
Committer: stack <st...@apache.org>
Committed: Thu Jan 3 21:35:14 2019 -0800

----------------------------------------------------------------------
 src/site/xdoc/downloads.xml | 13 ++++++-------
 1 file changed, 6 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/c3005653/src/site/xdoc/downloads.xml
----------------------------------------------------------------------
diff --git a/src/site/xdoc/downloads.xml b/src/site/xdoc/downloads.xml
index 4bb7f51..2859779 100644
--- a/src/site/xdoc/downloads.xml
+++ b/src/site/xdoc/downloads.xml
@@ -32,7 +32,6 @@ under the License.
   <a href="https://www.apache.org/dyn/closer.cgi#verify">Verify The Integrity Of The Files</a> for
   how to verify your mirrored downloads.
   </p>
-  <p style="color:red;"><strong>NOTE: 2.1.1 and 2.0.3 have a serious memory leak. See HBASE-21551. We are working on replacement releases.</strong></p>
   <section name="Releases">
   <table>
     <tr>
@@ -67,23 +66,23 @@ under the License.
     </tr>
     <tr>
       <td style="test-align: left">
-        2.0.3
+        2.0.4
       </td>
       <td style="test-align: left">
         2018/12/02
       </td>
       <td style="test-align: left">
-        <a href="https://apache.org/dist/hbase/2.0.3/compat-check-report-2.0.2-vs-2.0.3.html">2.0.2 vs 2.0.3</a>
+        <a href="https://apache.org/dist/hbase/2.0.4/compatibility_report_2.0.3vs2.0.4.html">2.0.3 vs 2.0.4</a>
       </td>
       <td style="test-align: left">
-        <a href="https://apache.org/dist/hbase/2.0.3/CHANGES.md">Changes</a>
+        <a href="https://apache.org/dist/hbase/2.0.4/CHANGES.md">Changes</a>
       </td>
       <td style="test-align: left">
-        <a href="https://apache.org/dist/hbase/2.0.3/RELEASENOTES.md">Release Notes</a>
+        <a href="https://apache.org/dist/hbase/2.0.4/RELEASENOTES.md">Release Notes</a>
       </td>
       <td style="test-align: left">
-        <a href="https://apache.org/dist/hbase/2.0.3/hbase-2.0.3-src.tar.gz">src</a> (<a href="https://apache.org/dist/hbase/2.0.3/hbase-2.0.3-src.tar.gz.sha512">sha512</a> <a href="https://apache.org/dist/hbase/2.0.3/hbase-2.0.3-src.tar.gz.asc">asc</a>) <br />
-        <a href="https://apache.org/dist/hbase/2.0.3/hbase-2.0.3-bin.tar.gz">bin</a> (<a href="https://apache.org/dist/hbase/2.0.3/hbase-2.0.3-src.tar.gz.sha512">sha512</a> <a href="https://apache.org/dist/hbase/2.0.3/hbase-2.0.3-src.tar.gz.asc">asc</a>)
+        <a href="https://apache.org/dist/hbase/2.0.4/hbase-2.0.4-src.tar.gz">src</a> (<a href="https://apache.org/dist/hbase/2.0.4/hbase-2.0.4-src.tar.gz.sha512">sha512</a> <a href="https://apache.org/dist/hbase/2.0.4/hbase-2.0.4-src.tar.gz.asc">asc</a>) <br />
+        <a href="https://apache.org/dist/hbase/2.0.4/hbase-2.0.4-bin.tar.gz">bin</a> (<a href="https://apache.org/dist/hbase/2.0.4/hbase-2.0.4-src.tar.gz.sha512">sha512</a> <a href="https://apache.org/dist/hbase/2.0.4/hbase-2.0.4-src.tar.gz.asc">asc</a>)
       </td>
     </tr>
     <tr>


[07/15] hbase git commit: HBASE-17356 Add replica get support

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/db66e6cc/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
index 4f73909..869a630 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hbase.client;
 
 import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME;
+import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
 
 import com.google.protobuf.Message;
 import com.google.protobuf.RpcChannel;
@@ -491,23 +492,23 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   @Override
   public CompletableFuture<TableDescriptor> getDescriptor(TableName tableName) {
     CompletableFuture<TableDescriptor> future = new CompletableFuture<>();
-    this.<List<TableSchema>> newMasterCaller()
-        .action(
-          (controller, stub) -> this
-              .<GetTableDescriptorsRequest, GetTableDescriptorsResponse, List<TableSchema>> call(
-                controller, stub, RequestConverter.buildGetTableDescriptorsRequest(tableName), (s,
-                    c, req, done) -> s.getTableDescriptors(c, req, done), (resp) -> resp
-                    .getTableSchemaList())).call().whenComplete((tableSchemas, error) -> {
-          if (error != null) {
-            future.completeExceptionally(error);
-            return;
-          }
-          if (!tableSchemas.isEmpty()) {
-            future.complete(ProtobufUtil.toTableDescriptor(tableSchemas.get(0)));
-          } else {
-            future.completeExceptionally(new TableNotFoundException(tableName.getNameAsString()));
-          }
-        });
+    addListener(this.<List<TableSchema>> newMasterCaller()
+      .action((controller, stub) -> this
+        .<GetTableDescriptorsRequest, GetTableDescriptorsResponse, List<TableSchema>> call(
+          controller, stub, RequestConverter.buildGetTableDescriptorsRequest(tableName),
+          (s, c, req, done) -> s.getTableDescriptors(c, req, done),
+          (resp) -> resp.getTableSchemaList()))
+      .call(), (tableSchemas, error) -> {
+        if (error != null) {
+          future.completeExceptionally(error);
+          return;
+        }
+        if (!tableSchemas.isEmpty()) {
+          future.complete(ProtobufUtil.toTableDescriptor(tableSchemas.get(0)));
+        } else {
+          future.completeExceptionally(new TableNotFoundException(tableName.getNameAsString()));
+        }
+      });
     return future;
   }
 
@@ -590,7 +591,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   @Override
   public CompletableFuture<Boolean> isTableEnabled(TableName tableName) {
     CompletableFuture<Boolean> future = new CompletableFuture<>();
-    AsyncMetaTableAccessor.getTableState(metaTable, tableName).whenComplete((state, error) -> {
+    addListener(AsyncMetaTableAccessor.getTableState(metaTable, tableName), (state, error) -> {
       if (error != null) {
         future.completeExceptionally(error);
         return;
@@ -607,7 +608,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   @Override
   public CompletableFuture<Boolean> isTableDisabled(TableName tableName) {
     CompletableFuture<Boolean> future = new CompletableFuture<>();
-    AsyncMetaTableAccessor.getTableState(metaTable, tableName).whenComplete((state, error) -> {
+    addListener(AsyncMetaTableAccessor.getTableState(metaTable, tableName), (state, error) -> {
       if (error != null) {
         future.completeExceptionally(error);
         return;
@@ -636,40 +637,37 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   private CompletableFuture<Boolean> isTableAvailable(TableName tableName,
       Optional<byte[][]> splitKeys) {
     CompletableFuture<Boolean> future = new CompletableFuture<>();
-    isTableEnabled(tableName).whenComplete(
-      (enabled, error) -> {
-        if (error != null) {
-          future.completeExceptionally(error);
-          return;
-        }
-        if (!enabled) {
-          future.complete(false);
-        } else {
-          AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName))
-              .whenComplete(
-                (locations, error1) -> {
-                  if (error1 != null) {
-                    future.completeExceptionally(error1);
-                    return;
-                  }
-                  List<HRegionLocation> notDeployedRegions =
-                      locations.stream().filter(loc -> loc.getServerName() == null)
-                          .collect(Collectors.toList());
-                  if (notDeployedRegions.size() > 0) {
-                    if (LOG.isDebugEnabled()) {
-                      LOG.debug("Table " + tableName + " has " + notDeployedRegions.size()
-                          + " regions");
-                    }
-                    future.complete(false);
-                    return;
-                  }
+    addListener(isTableEnabled(tableName), (enabled, error) -> {
+      if (error != null) {
+        future.completeExceptionally(error);
+        return;
+      }
+      if (!enabled) {
+        future.complete(false);
+      } else {
+        addListener(
+          AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName)),
+          (locations, error1) -> {
+            if (error1 != null) {
+              future.completeExceptionally(error1);
+              return;
+            }
+            List<HRegionLocation> notDeployedRegions = locations.stream()
+              .filter(loc -> loc.getServerName() == null).collect(Collectors.toList());
+            if (notDeployedRegions.size() > 0) {
+              if (LOG.isDebugEnabled()) {
+                LOG.debug("Table " + tableName + " has " + notDeployedRegions.size() + " regions");
+              }
+              future.complete(false);
+              return;
+            }
 
-                  Optional<Boolean> available =
-                      splitKeys.map(keys -> compareRegionsWithSplitKeys(locations, keys));
-                  future.complete(available.orElse(true));
-                });
-        }
-      });
+            Optional<Boolean> available =
+              splitKeys.map(keys -> compareRegionsWithSplitKeys(locations, keys));
+            future.complete(available.orElse(true));
+          });
+      }
+    });
     return future;
   }
 
@@ -791,20 +789,20 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   @Override
   public CompletableFuture<Void> flush(TableName tableName) {
     CompletableFuture<Void> future = new CompletableFuture<>();
-    tableExists(tableName).whenComplete((exists, err) -> {
+    addListener(tableExists(tableName), (exists, err) -> {
       if (err != null) {
         future.completeExceptionally(err);
       } else if (!exists) {
         future.completeExceptionally(new TableNotFoundException(tableName));
       } else {
-        isTableEnabled(tableName).whenComplete((tableEnabled, err2) -> {
+        addListener(isTableEnabled(tableName), (tableEnabled, err2) -> {
           if (err2 != null) {
             future.completeExceptionally(err2);
           } else if (!tableEnabled) {
             future.completeExceptionally(new TableNotEnabledException(tableName));
           } else {
-            execProcedure(FLUSH_TABLE_PROCEDURE_SIGNATURE, tableName.getNameAsString(),
-              new HashMap<>()).whenComplete((ret, err3) -> {
+            addListener(execProcedure(FLUSH_TABLE_PROCEDURE_SIGNATURE, tableName.getNameAsString(),
+              new HashMap<>()), (ret, err3) -> {
                 if (err3 != null) {
                   future.completeExceptionally(err3);
                 } else {
@@ -821,27 +819,25 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   @Override
   public CompletableFuture<Void> flushRegion(byte[] regionName) {
     CompletableFuture<Void> future = new CompletableFuture<>();
-    getRegionLocation(regionName).whenComplete(
-      (location, err) -> {
-        if (err != null) {
-          future.completeExceptionally(err);
-          return;
-        }
-        ServerName serverName = location.getServerName();
-        if (serverName == null) {
-          future.completeExceptionally(new NoServerForRegionException(Bytes
-              .toStringBinary(regionName)));
-          return;
+    addListener(getRegionLocation(regionName), (location, err) -> {
+      if (err != null) {
+        future.completeExceptionally(err);
+        return;
+      }
+      ServerName serverName = location.getServerName();
+      if (serverName == null) {
+        future
+          .completeExceptionally(new NoServerForRegionException(Bytes.toStringBinary(regionName)));
+        return;
+      }
+      addListener(flush(serverName, location.getRegion()), (ret, err2) -> {
+        if (err2 != null) {
+          future.completeExceptionally(err2);
+        } else {
+          future.complete(ret);
         }
-        flush(serverName, location.getRegion())
-          .whenComplete((ret, err2) -> {
-              if (err2 != null) {
-                future.completeExceptionally(err2);
-              } else {
-                future.complete(ret);
-              }
-          });
       });
+    });
     return future;
   }
 
@@ -859,7 +855,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   @Override
   public CompletableFuture<Void> flushRegionServer(ServerName sn) {
     CompletableFuture<Void> future = new CompletableFuture<>();
-    getRegions(sn).whenComplete((hRegionInfos, err) -> {
+    addListener(getRegions(sn), (hRegionInfos, err) -> {
       if (err != null) {
         future.completeExceptionally(err);
         return;
@@ -868,9 +864,8 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
       if (hRegionInfos != null) {
         hRegionInfos.forEach(region -> compactFutures.add(flush(sn, region)));
       }
-      CompletableFuture
-        .allOf(compactFutures.toArray(new CompletableFuture<?>[compactFutures.size()]))
-        .whenComplete((ret, err2) -> {
+      addListener(CompletableFuture.allOf(
+        compactFutures.toArray(new CompletableFuture<?>[compactFutures.size()])), (ret, err2) -> {
           if (err2 != null) {
             future.completeExceptionally(err2);
           } else {
@@ -943,7 +938,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
 
   private CompletableFuture<Void> compactRegionServer(ServerName sn, boolean major) {
     CompletableFuture<Void> future = new CompletableFuture<>();
-    getRegions(sn).whenComplete((hRegionInfos, err) -> {
+    addListener(getRegions(sn), (hRegionInfos, err) -> {
       if (err != null) {
         future.completeExceptionally(err);
         return;
@@ -952,15 +947,14 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
       if (hRegionInfos != null) {
         hRegionInfos.forEach(region -> compactFutures.add(compact(sn, region, major, null)));
       }
-      CompletableFuture
-          .allOf(compactFutures.toArray(new CompletableFuture<?>[compactFutures.size()]))
-          .whenComplete((ret, err2) -> {
-            if (err2 != null) {
-              future.completeExceptionally(err2);
-            } else {
-              future.complete(ret);
-            }
-          });
+      addListener(CompletableFuture.allOf(
+        compactFutures.toArray(new CompletableFuture<?>[compactFutures.size()])), (ret, err2) -> {
+          if (err2 != null) {
+            future.completeExceptionally(err2);
+          } else {
+            future.complete(ret);
+          }
+        });
     });
     return future;
   }
@@ -968,28 +962,26 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   private CompletableFuture<Void> compactRegion(byte[] regionName, byte[] columnFamily,
       boolean major) {
     CompletableFuture<Void> future = new CompletableFuture<>();
-
-    getRegionLocation(regionName).whenComplete(
-      (location, err) -> {
-        if (err != null) {
-          future.completeExceptionally(err);
-          return;
-        }
-        ServerName serverName = location.getServerName();
-        if (serverName == null) {
-          future.completeExceptionally(new NoServerForRegionException(Bytes
-              .toStringBinary(regionName)));
-          return;
-        }
-        compact(location.getServerName(), location.getRegion(), major, columnFamily)
-            .whenComplete((ret, err2) -> {
-              if (err2 != null) {
-                future.completeExceptionally(err2);
-              } else {
-                future.complete(ret);
-              }
-            });
-      });
+    addListener(getRegionLocation(regionName), (location, err) -> {
+      if (err != null) {
+        future.completeExceptionally(err);
+        return;
+      }
+      ServerName serverName = location.getServerName();
+      if (serverName == null) {
+        future
+          .completeExceptionally(new NoServerForRegionException(Bytes.toStringBinary(regionName)));
+        return;
+      }
+      addListener(compact(location.getServerName(), location.getRegion(), major, columnFamily),
+        (ret, err2) -> {
+          if (err2 != null) {
+            future.completeExceptionally(err2);
+          } else {
+            future.complete(ret);
+          }
+        });
+    });
     return future;
   }
 
@@ -1001,19 +993,18 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
       CompletableFuture<List<HRegionLocation>> future = new CompletableFuture<>();
       // For meta table, we use zk to fetch all locations.
       AsyncRegistry registry = AsyncRegistryFactory.getRegistry(connection.getConfiguration());
-      registry.getMetaRegionLocation().whenComplete(
-        (metaRegions, err) -> {
-          if (err != null) {
-            future.completeExceptionally(err);
-          } else if (metaRegions == null || metaRegions.isEmpty()
-              || metaRegions.getDefaultRegionLocation() == null) {
-            future.completeExceptionally(new IOException("meta region does not found"));
-          } else {
-            future.complete(Collections.singletonList(metaRegions.getDefaultRegionLocation()));
-          }
-          // close the registry.
-          IOUtils.closeQuietly(registry);
-        });
+      addListener(registry.getMetaRegionLocation(), (metaRegions, err) -> {
+        if (err != null) {
+          future.completeExceptionally(err);
+        } else if (metaRegions == null || metaRegions.isEmpty() ||
+          metaRegions.getDefaultRegionLocation() == null) {
+          future.completeExceptionally(new IOException("meta region does not found"));
+        } else {
+          future.complete(Collections.singletonList(metaRegions.getDefaultRegionLocation()));
+        }
+        // close the registry.
+        IOUtils.closeQuietly(registry);
+      });
       return future;
     } else {
       // For non-meta table, we fetch all locations by scanning hbase:meta table
@@ -1024,40 +1015,40 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   /**
    * Compact column family of a table, Asynchronous operation even if CompletableFuture.get()
    */
-  private CompletableFuture<Void> compact(TableName tableName, byte[] columnFamily,
-      boolean major, CompactType compactType) {
+  private CompletableFuture<Void> compact(TableName tableName, byte[] columnFamily, boolean major,
+      CompactType compactType) {
     CompletableFuture<Void> future = new CompletableFuture<>();
 
     switch (compactType) {
       case MOB:
-        connection.registry.getMasterAddress().whenComplete((serverName, err) -> {
+        addListener(connection.registry.getMasterAddress(), (serverName, err) -> {
           if (err != null) {
             future.completeExceptionally(err);
             return;
           }
           RegionInfo regionInfo = RegionInfo.createMobRegionInfo(tableName);
-          compact(serverName, regionInfo, major, columnFamily)
-              .whenComplete((ret, err2) -> {
-                if (err2 != null) {
-                  future.completeExceptionally(err2);
-                } else {
-                  future.complete(ret);
-                }
-              });
+          addListener(compact(serverName, regionInfo, major, columnFamily), (ret, err2) -> {
+            if (err2 != null) {
+              future.completeExceptionally(err2);
+            } else {
+              future.complete(ret);
+            }
+          });
         });
         break;
       case NORMAL:
-        getTableHRegionLocations(tableName).whenComplete((locations, err) -> {
+        addListener(getTableHRegionLocations(tableName), (locations, err) -> {
           if (err != null) {
             future.completeExceptionally(err);
             return;
           }
-          CompletableFuture<?>[] compactFutures = locations.stream().filter(l -> l.getRegion() != null)
+          CompletableFuture<?>[] compactFutures =
+            locations.stream().filter(l -> l.getRegion() != null)
               .filter(l -> !l.getRegion().isOffline()).filter(l -> l.getServerName() != null)
               .map(l -> compact(l.getServerName(), l.getRegion(), major, columnFamily))
               .toArray(CompletableFuture<?>[]::new);
           // future complete unless all of the compact futures are completed.
-          CompletableFuture.allOf(compactFutures).whenComplete((ret, err2) -> {
+          addListener(CompletableFuture.allOf(compactFutures), (ret, err2) -> {
             if (err2 != null) {
               future.completeExceptionally(err2);
             } else {
@@ -1098,29 +1089,28 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
 
   private void checkAndGetTableName(byte[] encodeRegionName, AtomicReference<TableName> tableName,
       CompletableFuture<TableName> result) {
-    getRegionLocation(encodeRegionName).whenComplete(
-      (location, err) -> {
-        if (err != null) {
-          result.completeExceptionally(err);
-          return;
-        }
-        RegionInfo regionInfo = location.getRegion();
-        if (regionInfo.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) {
-          result.completeExceptionally(new IllegalArgumentException(
-              "Can't invoke merge on non-default regions directly"));
-          return;
-        }
-        if (!tableName.compareAndSet(null, regionInfo.getTable())) {
-          if (!tableName.get().equals(regionInfo.getTable())) {
-            // tables of this two region should be same.
-            result.completeExceptionally(new IllegalArgumentException(
-                "Cannot merge regions from two different tables " + tableName.get() + " and "
-                    + regionInfo.getTable()));
-          } else {
-            result.complete(tableName.get());
-          }
+    addListener(getRegionLocation(encodeRegionName), (location, err) -> {
+      if (err != null) {
+        result.completeExceptionally(err);
+        return;
+      }
+      RegionInfo regionInfo = location.getRegion();
+      if (regionInfo.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) {
+        result.completeExceptionally(
+          new IllegalArgumentException("Can't invoke merge on non-default regions directly"));
+        return;
+      }
+      if (!tableName.compareAndSet(null, regionInfo.getTable())) {
+        if (!tableName.get().equals(regionInfo.getTable())) {
+          // tables of this two region should be same.
+          result.completeExceptionally(
+            new IllegalArgumentException("Cannot merge regions from two different tables " +
+              tableName.get() + " and " + regionInfo.getTable()));
+        } else {
+          result.complete(tableName.get());
         }
-      });
+      }
+    });
   }
 
   private CompletableFuture<TableName> checkRegionsAndGetTableName(byte[] encodeRegionNameA,
@@ -1185,41 +1175,42 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
     final byte[] encodeRegionNameA = toEncodeRegionName(nameOfRegionA);
     final byte[] encodeRegionNameB = toEncodeRegionName(nameOfRegionB);
 
-    checkRegionsAndGetTableName(encodeRegionNameA, encodeRegionNameB)
-        .whenComplete((tableName, err) -> {
-          if (err != null) {
-            future.completeExceptionally(err);
-            return;
-          }
+    addListener(checkRegionsAndGetTableName(encodeRegionNameA, encodeRegionNameB),
+      (tableName, err) -> {
+        if (err != null) {
+          future.completeExceptionally(err);
+          return;
+        }
 
-          MergeTableRegionsRequest request = null;
-          try {
-            request = RequestConverter.buildMergeTableRegionsRequest(
-              new byte[][] { encodeRegionNameA, encodeRegionNameB }, forcible, ng.getNonceGroup(),
-              ng.newNonce());
-          } catch (DeserializationException e) {
-            future.completeExceptionally(e);
-            return;
-          }
+        MergeTableRegionsRequest request = null;
+        try {
+          request = RequestConverter.buildMergeTableRegionsRequest(
+            new byte[][] { encodeRegionNameA, encodeRegionNameB }, forcible, ng.getNonceGroup(),
+            ng.newNonce());
+        } catch (DeserializationException e) {
+          future.completeExceptionally(e);
+          return;
+        }
 
+        addListener(
           this.<MergeTableRegionsRequest, MergeTableRegionsResponse> procedureCall(request,
             (s, c, req, done) -> s.mergeTableRegions(c, req, done), (resp) -> resp.getProcId(),
-            new MergeTableRegionProcedureBiConsumer(tableName)).whenComplete((ret, err2) -> {
-              if (err2 != null) {
-                future.completeExceptionally(err2);
-              } else {
-                future.complete(ret);
-              }
-            });
-
-        });
+            new MergeTableRegionProcedureBiConsumer(tableName)),
+          (ret, err2) -> {
+            if (err2 != null) {
+              future.completeExceptionally(err2);
+            } else {
+              future.complete(ret);
+            }
+          });
+      });
     return future;
   }
 
   @Override
   public CompletableFuture<Void> split(TableName tableName) {
     CompletableFuture<Void> future = new CompletableFuture<>();
-    tableExists(tableName).whenComplete((exist, error) -> {
+    addListener(tableExists(tableName), (exist, error) -> {
       if (error != null) {
         future.completeExceptionally(error);
         return;
@@ -1228,45 +1219,50 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
         future.completeExceptionally(new TableNotFoundException(tableName));
         return;
       }
-      metaTable
+      addListener(
+        metaTable
           .scanAll(new Scan().setReadType(ReadType.PREAD).addFamily(HConstants.CATALOG_FAMILY)
-              .withStartRow(MetaTableAccessor.getTableStartRowForMeta(tableName, QueryType.REGION))
-              .withStopRow(MetaTableAccessor.getTableStopRowForMeta(tableName, QueryType.REGION)))
-          .whenComplete((results, err2) -> {
-            if (err2 != null) {
-              future.completeExceptionally(err2);
-              return;
-            }
-            if (results != null && !results.isEmpty()) {
-              List<CompletableFuture<Void>> splitFutures = new ArrayList<>();
-              for (Result r : results) {
-                if (r.isEmpty() || MetaTableAccessor.getRegionInfo(r) == null) continue;
-                RegionLocations rl = MetaTableAccessor.getRegionLocations(r);
-                if (rl != null) {
-                  for (HRegionLocation h : rl.getRegionLocations()) {
-                    if (h != null && h.getServerName() != null) {
-                      RegionInfo hri = h.getRegion();
-                      if (hri == null || hri.isSplitParent()
-                          || hri.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID)
-                        continue;
-                      splitFutures.add(split(hri, null));
+            .withStartRow(MetaTableAccessor.getTableStartRowForMeta(tableName, QueryType.REGION))
+            .withStopRow(MetaTableAccessor.getTableStopRowForMeta(tableName, QueryType.REGION))),
+        (results, err2) -> {
+          if (err2 != null) {
+            future.completeExceptionally(err2);
+            return;
+          }
+          if (results != null && !results.isEmpty()) {
+            List<CompletableFuture<Void>> splitFutures = new ArrayList<>();
+            for (Result r : results) {
+              if (r.isEmpty() || MetaTableAccessor.getRegionInfo(r) == null) {
+                continue;
+              }
+              RegionLocations rl = MetaTableAccessor.getRegionLocations(r);
+              if (rl != null) {
+                for (HRegionLocation h : rl.getRegionLocations()) {
+                  if (h != null && h.getServerName() != null) {
+                    RegionInfo hri = h.getRegion();
+                    if (hri == null || hri.isSplitParent() ||
+                      hri.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) {
+                      continue;
                     }
+                    splitFutures.add(split(hri, null));
                   }
                 }
               }
-              CompletableFuture
-                  .allOf(splitFutures.toArray(new CompletableFuture<?>[splitFutures.size()]))
-                  .whenComplete((ret, exception) -> {
-                    if (exception != null) {
-                      future.completeExceptionally(exception);
-                      return;
-                    }
-                    future.complete(ret);
-                  });
-            } else {
-              future.complete(null);
             }
-          });
+            addListener(
+              CompletableFuture
+                .allOf(splitFutures.toArray(new CompletableFuture<?>[splitFutures.size()])),
+              (ret, exception) -> {
+                if (exception != null) {
+                  future.completeExceptionally(exception);
+                  return;
+                }
+                future.complete(ret);
+              });
+          } else {
+            future.complete(null);
+          }
+        });
     });
     return future;
   }
@@ -1277,54 +1273,52 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
     if (splitPoint == null) {
       return failedFuture(new IllegalArgumentException("splitPoint can not be null."));
     }
-    connection.getRegionLocator(tableName).getRegionLocation(splitPoint)
-        .whenComplete((loc, err) -> {
-          if (err != null) {
-            result.completeExceptionally(err);
-          } else if (loc == null || loc.getRegion() == null) {
-            result.completeExceptionally(new IllegalArgumentException(
-                "Region does not found: rowKey=" + Bytes.toStringBinary(splitPoint)));
-          } else {
-            splitRegion(loc.getRegion().getRegionName(), splitPoint)
-                .whenComplete((ret, err2) -> {
-                  if (err2 != null) {
-                    result.completeExceptionally(err2);
-                  } else {
-                    result.complete(ret);
-                  }
+    addListener(connection.getRegionLocator(tableName).getRegionLocation(splitPoint),
+      (loc, err) -> {
+        if (err != null) {
+          result.completeExceptionally(err);
+        } else if (loc == null || loc.getRegion() == null) {
+          result.completeExceptionally(new IllegalArgumentException(
+            "Region does not found: rowKey=" + Bytes.toStringBinary(splitPoint)));
+        } else {
+          addListener(splitRegion(loc.getRegion().getRegionName(), splitPoint), (ret, err2) -> {
+            if (err2 != null) {
+              result.completeExceptionally(err2);
+            } else {
+              result.complete(ret);
+            }
 
-                });
-          }
-        });
+          });
+        }
+      });
     return result;
   }
 
   @Override
   public CompletableFuture<Void> splitRegion(byte[] regionName) {
     CompletableFuture<Void> future = new CompletableFuture<>();
-    getRegionLocation(regionName).whenComplete(
-      (location, err) -> {
-        RegionInfo regionInfo = location.getRegion();
-        if (regionInfo.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) {
-          future.completeExceptionally(new IllegalArgumentException(
-              "Can't split replicas directly. "
-                  + "Replicas are auto-split when their primary is split."));
-          return;
-        }
-        ServerName serverName = location.getServerName();
-        if (serverName == null) {
-          future.completeExceptionally(new NoServerForRegionException(Bytes
-              .toStringBinary(regionName)));
-          return;
+    addListener(getRegionLocation(regionName), (location, err) -> {
+      RegionInfo regionInfo = location.getRegion();
+      if (regionInfo.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) {
+        future
+          .completeExceptionally(new IllegalArgumentException("Can't split replicas directly. " +
+            "Replicas are auto-split when their primary is split."));
+        return;
+      }
+      ServerName serverName = location.getServerName();
+      if (serverName == null) {
+        future
+          .completeExceptionally(new NoServerForRegionException(Bytes.toStringBinary(regionName)));
+        return;
+      }
+      addListener(split(regionInfo, null), (ret, err2) -> {
+        if (err2 != null) {
+          future.completeExceptionally(err2);
+        } else {
+          future.complete(ret);
         }
-        split(regionInfo, null).whenComplete((ret, err2) -> {
-          if (err2 != null) {
-            future.completeExceptionally(err2);
-          } else {
-            future.complete(ret);
-          }
-        });
       });
+    });
     return future;
   }
 
@@ -1333,35 +1327,34 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
     Preconditions.checkNotNull(splitPoint,
       "splitPoint is null. If you don't specify a splitPoint, use splitRegion(byte[]) instead");
     CompletableFuture<Void> future = new CompletableFuture<>();
-    getRegionLocation(regionName).whenComplete(
-      (location, err) -> {
-        RegionInfo regionInfo = location.getRegion();
-        if (regionInfo.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) {
-          future.completeExceptionally(new IllegalArgumentException(
-              "Can't split replicas directly. "
-                  + "Replicas are auto-split when their primary is split."));
-          return;
-        }
-        ServerName serverName = location.getServerName();
-        if (serverName == null) {
-          future.completeExceptionally(new NoServerForRegionException(Bytes
-              .toStringBinary(regionName)));
-          return;
-        }
-        if (regionInfo.getStartKey() != null
-            && Bytes.compareTo(regionInfo.getStartKey(), splitPoint) == 0) {
-          future.completeExceptionally(new IllegalArgumentException(
-              "should not give a splitkey which equals to startkey!"));
-          return;
+    addListener(getRegionLocation(regionName), (location, err) -> {
+      RegionInfo regionInfo = location.getRegion();
+      if (regionInfo.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) {
+        future
+          .completeExceptionally(new IllegalArgumentException("Can't split replicas directly. " +
+            "Replicas are auto-split when their primary is split."));
+        return;
+      }
+      ServerName serverName = location.getServerName();
+      if (serverName == null) {
+        future
+          .completeExceptionally(new NoServerForRegionException(Bytes.toStringBinary(regionName)));
+        return;
+      }
+      if (regionInfo.getStartKey() != null &&
+        Bytes.compareTo(regionInfo.getStartKey(), splitPoint) == 0) {
+        future.completeExceptionally(
+          new IllegalArgumentException("should not give a splitkey which equals to startkey!"));
+        return;
+      }
+      addListener(split(regionInfo, splitPoint), (ret, err2) -> {
+        if (err2 != null) {
+          future.completeExceptionally(err2);
+        } else {
+          future.complete(ret);
         }
-        split(regionInfo, splitPoint).whenComplete((ret, err2) -> {
-          if (err2 != null) {
-            future.completeExceptionally(err2);
-          } else {
-            future.complete(ret);
-          }
-        });
       });
+    });
     return future;
   }
 
@@ -1370,121 +1363,119 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
     TableName tableName = hri.getTable();
     SplitTableRegionRequest request = null;
     try {
-      request = RequestConverter
-          .buildSplitTableRegionRequest(hri, splitPoint,
-              ng.getNonceGroup(), ng.newNonce());
+      request = RequestConverter.buildSplitTableRegionRequest(hri, splitPoint, ng.getNonceGroup(),
+        ng.newNonce());
     } catch (DeserializationException e) {
       future.completeExceptionally(e);
       return future;
     }
 
-    this.<SplitTableRegionRequest, SplitTableRegionResponse>procedureCall(request,
-        (s, c, req, done) -> s.splitRegion(c, req, done), (resp) -> resp.getProcId(),
-        new SplitTableRegionProcedureBiConsumer(tableName)).whenComplete((ret, err2) -> {
-      if (err2 != null) {
-        future.completeExceptionally(err2);
-      } else {
-        future.complete(ret);
-      }
-    });
+    addListener(this.<SplitTableRegionRequest, SplitTableRegionResponse> procedureCall(request,
+      (s, c, req, done) -> s.splitRegion(c, req, done), (resp) -> resp.getProcId(),
+      new SplitTableRegionProcedureBiConsumer(tableName)), (ret, err2) -> {
+        if (err2 != null) {
+          future.completeExceptionally(err2);
+        } else {
+          future.complete(ret);
+        }
+      });
     return future;
   }
 
   @Override
   public CompletableFuture<Void> assign(byte[] regionName) {
     CompletableFuture<Void> future = new CompletableFuture<>();
-    getRegionInfo(regionName).whenComplete(
-      (regionInfo, err) -> {
-        if (err != null) {
-          future.completeExceptionally(err);
-          return;
-        }
-        this.<Void> newMasterCaller()
-            .action(
-              ((controller, stub) -> this.<AssignRegionRequest, AssignRegionResponse, Void> call(
-                controller, stub, RequestConverter.buildAssignRegionRequest(regionInfo
-                    .getRegionName()), (s, c, req, done) -> s.assignRegion(c, req, done),
-                resp -> null))).call().whenComplete((ret, err2) -> {
-              if (err2 != null) {
-                future.completeExceptionally(err2);
-              } else {
-                future.complete(ret);
-              }
-            });
-      });
+    addListener(getRegionInfo(regionName), (regionInfo, err) -> {
+      if (err != null) {
+        future.completeExceptionally(err);
+        return;
+      }
+      addListener(this.<Void> newMasterCaller()
+        .action(((controller, stub) -> this.<AssignRegionRequest, AssignRegionResponse, Void> call(
+          controller, stub, RequestConverter.buildAssignRegionRequest(regionInfo.getRegionName()),
+          (s, c, req, done) -> s.assignRegion(c, req, done), resp -> null)))
+        .call(), (ret, err2) -> {
+          if (err2 != null) {
+            future.completeExceptionally(err2);
+          } else {
+            future.complete(ret);
+          }
+        });
+    });
     return future;
   }
 
   @Override
   public CompletableFuture<Void> unassign(byte[] regionName, boolean forcible) {
     CompletableFuture<Void> future = new CompletableFuture<>();
-    getRegionInfo(regionName).whenComplete(
-      (regionInfo, err) -> {
-        if (err != null) {
-          future.completeExceptionally(err);
-          return;
-        }
+    addListener(getRegionInfo(regionName), (regionInfo, err) -> {
+      if (err != null) {
+        future.completeExceptionally(err);
+        return;
+      }
+      addListener(
         this.<Void> newMasterCaller()
-            .action(
-              ((controller, stub) -> this
-                  .<UnassignRegionRequest, UnassignRegionResponse, Void> call(controller, stub,
-                    RequestConverter.buildUnassignRegionRequest(regionInfo.getRegionName(), forcible),
-                    (s, c, req, done) -> s.unassignRegion(c, req, done), resp -> null))).call()
-            .whenComplete((ret, err2) -> {
-              if (err2 != null) {
-                future.completeExceptionally(err2);
-              } else {
-                future.complete(ret);
-              }
-            });
-      });
+          .action(((controller, stub) -> this
+            .<UnassignRegionRequest, UnassignRegionResponse, Void> call(controller, stub,
+              RequestConverter.buildUnassignRegionRequest(regionInfo.getRegionName(), forcible),
+              (s, c, req, done) -> s.unassignRegion(c, req, done), resp -> null)))
+          .call(),
+        (ret, err2) -> {
+          if (err2 != null) {
+            future.completeExceptionally(err2);
+          } else {
+            future.complete(ret);
+          }
+        });
+    });
     return future;
   }
 
   @Override
   public CompletableFuture<Void> offline(byte[] regionName) {
     CompletableFuture<Void> future = new CompletableFuture<>();
-    getRegionInfo(regionName).whenComplete(
-      (regionInfo, err) -> {
-        if (err != null) {
-          future.completeExceptionally(err);
-          return;
-        }
+    addListener(getRegionInfo(regionName), (regionInfo, err) -> {
+      if (err != null) {
+        future.completeExceptionally(err);
+        return;
+      }
+      addListener(
         this.<Void> newMasterCaller()
-            .action(
-              ((controller, stub) -> this.<OfflineRegionRequest, OfflineRegionResponse, Void> call(
-                controller, stub, RequestConverter.buildOfflineRegionRequest(regionInfo
-                    .getRegionName()), (s, c, req, done) -> s.offlineRegion(c, req, done),
-                resp -> null))).call().whenComplete((ret, err2) -> {
-              if (err2 != null) {
-                future.completeExceptionally(err2);
-              } else {
-                future.complete(ret);
-              }
-            });
-      });
+          .action(((controller, stub) -> this
+            .<OfflineRegionRequest, OfflineRegionResponse, Void> call(controller, stub,
+              RequestConverter.buildOfflineRegionRequest(regionInfo.getRegionName()),
+              (s, c, req, done) -> s.offlineRegion(c, req, done), resp -> null)))
+          .call(),
+        (ret, err2) -> {
+          if (err2 != null) {
+            future.completeExceptionally(err2);
+          } else {
+            future.complete(ret);
+          }
+        });
+    });
     return future;
   }
 
   @Override
   public CompletableFuture<Void> move(byte[] regionName) {
     CompletableFuture<Void> future = new CompletableFuture<>();
-    getRegionInfo(regionName).whenComplete(
-      (regionInfo, err) -> {
-        if (err != null) {
-          future.completeExceptionally(err);
-          return;
-        }
+    addListener(getRegionInfo(regionName), (regionInfo, err) -> {
+      if (err != null) {
+        future.completeExceptionally(err);
+        return;
+      }
+      addListener(
         moveRegion(
-          RequestConverter.buildMoveRegionRequest(regionInfo.getEncodedNameAsBytes(), null))
-            .whenComplete((ret, err2) -> {
-              if (err2 != null) {
-                future.completeExceptionally(err2);
-              } else {
-                future.complete(ret);
-              }
-            });
-      });
+          RequestConverter.buildMoveRegionRequest(regionInfo.getEncodedNameAsBytes(), null)),
+        (ret, err2) -> {
+          if (err2 != null) {
+            future.completeExceptionally(err2);
+          } else {
+            future.complete(ret);
+          }
+        });
+    });
     return future;
   }
 
@@ -1493,20 +1484,20 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
     Preconditions.checkNotNull(destServerName,
       "destServerName is null. If you don't specify a destServerName, use move(byte[]) instead");
     CompletableFuture<Void> future = new CompletableFuture<>();
-    getRegionInfo(regionName).whenComplete((regionInfo, err) -> {
+    addListener(getRegionInfo(regionName), (regionInfo, err) -> {
       if (err != null) {
         future.completeExceptionally(err);
         return;
       }
-      moveRegion(
-        RequestConverter.buildMoveRegionRequest(regionInfo.getEncodedNameAsBytes(), destServerName))
-            .whenComplete((ret, err2) -> {
-              if (err2 != null) {
-                future.completeExceptionally(err2);
-              } else {
-                future.complete(ret);
-              }
-            });
+      addListener(moveRegion(RequestConverter
+        .buildMoveRegionRequest(regionInfo.getEncodedNameAsBytes(), destServerName)),
+        (ret, err2) -> {
+          if (err2 != null) {
+            future.completeExceptionally(err2);
+          } else {
+            future.complete(ret);
+          }
+        });
     });
     return future;
   }
@@ -1636,11 +1627,11 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
     }
 
     CompletableFuture<Void> future = new CompletableFuture<Void>();
-    getReplicationPeerConfig(id).whenComplete((peerConfig, error) -> {
+    addListener(getReplicationPeerConfig(id), (peerConfig, error) -> {
       if (!completeExceptionally(future, error)) {
         ReplicationPeerConfig newPeerConfig =
-            ReplicationPeerConfigUtil.appendTableCFsToReplicationPeerConfig(tableCfs, peerConfig);
-        updateReplicationPeerConfig(id, newPeerConfig).whenComplete((result, err) -> {
+          ReplicationPeerConfigUtil.appendTableCFsToReplicationPeerConfig(tableCfs, peerConfig);
+        addListener(updateReplicationPeerConfig(id, newPeerConfig), (result, err) -> {
           if (!completeExceptionally(future, error)) {
             future.complete(result);
           }
@@ -1658,24 +1649,23 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
     }
 
     CompletableFuture<Void> future = new CompletableFuture<Void>();
-    getReplicationPeerConfig(id).whenComplete(
-      (peerConfig, error) -> {
-        if (!completeExceptionally(future, error)) {
-          ReplicationPeerConfig newPeerConfig = null;
-          try {
-            newPeerConfig = ReplicationPeerConfigUtil
-                .removeTableCFsFromReplicationPeerConfig(tableCfs, peerConfig, id);
-          } catch (ReplicationException e) {
-            future.completeExceptionally(e);
-            return;
-          }
-          updateReplicationPeerConfig(id, newPeerConfig).whenComplete((result, err) -> {
-            if (!completeExceptionally(future, error)) {
-              future.complete(result);
-            }
-          });
+    addListener(getReplicationPeerConfig(id), (peerConfig, error) -> {
+      if (!completeExceptionally(future, error)) {
+        ReplicationPeerConfig newPeerConfig = null;
+        try {
+          newPeerConfig = ReplicationPeerConfigUtil
+            .removeTableCFsFromReplicationPeerConfig(tableCfs, peerConfig, id);
+        } catch (ReplicationException e) {
+          future.completeExceptionally(e);
+          return;
         }
-      });
+        addListener(updateReplicationPeerConfig(id, newPeerConfig), (result, err) -> {
+          if (!completeExceptionally(future, error)) {
+            future.complete(result);
+          }
+        });
+      }
+    });
     return future;
   }
 
@@ -1710,31 +1700,30 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   @Override
   public CompletableFuture<List<TableCFs>> listReplicatedTableCFs() {
     CompletableFuture<List<TableCFs>> future = new CompletableFuture<List<TableCFs>>();
-    listTableDescriptors().whenComplete(
-      (tables, error) -> {
-        if (!completeExceptionally(future, error)) {
-          List<TableCFs> replicatedTableCFs = new ArrayList<>();
-          tables.forEach(table -> {
-            Map<String, Integer> cfs = new HashMap<>();
-            Stream.of(table.getColumnFamilies())
-                .filter(column -> column.getScope() != HConstants.REPLICATION_SCOPE_LOCAL)
-                .forEach(column -> {
-                  cfs.put(column.getNameAsString(), column.getScope());
-                });
-            if (!cfs.isEmpty()) {
-              replicatedTableCFs.add(new TableCFs(table.getTableName(), cfs));
-            }
-          });
-          future.complete(replicatedTableCFs);
-        }
-      });
+    addListener(listTableDescriptors(), (tables, error) -> {
+      if (!completeExceptionally(future, error)) {
+        List<TableCFs> replicatedTableCFs = new ArrayList<>();
+        tables.forEach(table -> {
+          Map<String, Integer> cfs = new HashMap<>();
+          Stream.of(table.getColumnFamilies())
+            .filter(column -> column.getScope() != HConstants.REPLICATION_SCOPE_LOCAL)
+            .forEach(column -> {
+              cfs.put(column.getNameAsString(), column.getScope());
+            });
+          if (!cfs.isEmpty()) {
+            replicatedTableCFs.add(new TableCFs(table.getTableName(), cfs));
+          }
+        });
+        future.complete(replicatedTableCFs);
+      }
+    });
     return future;
   }
 
   @Override
   public CompletableFuture<Void> snapshot(SnapshotDescription snapshotDesc) {
-    SnapshotProtos.SnapshotDescription snapshot = ProtobufUtil
-        .createHBaseProtosSnapshotDesc(snapshotDesc);
+    SnapshotProtos.SnapshotDescription snapshot =
+      ProtobufUtil.createHBaseProtosSnapshotDesc(snapshotDesc);
     try {
       ClientSnapshotDescriptionUtils.assertSnapshotRequestIsValid(snapshot);
     } catch (IllegalArgumentException e) {
@@ -1742,47 +1731,47 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
     }
     CompletableFuture<Void> future = new CompletableFuture<>();
     final SnapshotRequest request = SnapshotRequest.newBuilder().setSnapshot(snapshot).build();
-    this.<Long> newMasterCaller()
-        .action(
-          (controller, stub) -> this.<SnapshotRequest, SnapshotResponse, Long> call(controller,
-            stub, request, (s, c, req, done) -> s.snapshot(c, req, done),
-            resp -> resp.getExpectedTimeout())).call().whenComplete((expectedTimeout, err) -> {
-          if (err != null) {
-            future.completeExceptionally(err);
-            return;
-          }
-          TimerTask pollingTask = new TimerTask() {
-            int tries = 0;
-            long startTime = EnvironmentEdgeManager.currentTime();
-            long endTime = startTime + expectedTimeout;
-            long maxPauseTime = expectedTimeout / maxAttempts;
-
-            @Override
-            public void run(Timeout timeout) throws Exception {
-              if (EnvironmentEdgeManager.currentTime() < endTime) {
-                isSnapshotFinished(snapshotDesc).whenComplete((done, err2) -> {
-                  if (err2 != null) {
-                    future.completeExceptionally(err2);
-                  } else if (done) {
-                    future.complete(null);
-                  } else {
-                    // retry again after pauseTime.
-                  long pauseTime = ConnectionUtils.getPauseTime(
-                    TimeUnit.NANOSECONDS.toMillis(pauseNs), ++tries);
+    addListener(this.<Long> newMasterCaller()
+      .action((controller, stub) -> this.<SnapshotRequest, SnapshotResponse, Long> call(controller,
+        stub, request, (s, c, req, done) -> s.snapshot(c, req, done),
+        resp -> resp.getExpectedTimeout()))
+      .call(), (expectedTimeout, err) -> {
+        if (err != null) {
+          future.completeExceptionally(err);
+          return;
+        }
+        TimerTask pollingTask = new TimerTask() {
+          int tries = 0;
+          long startTime = EnvironmentEdgeManager.currentTime();
+          long endTime = startTime + expectedTimeout;
+          long maxPauseTime = expectedTimeout / maxAttempts;
+
+          @Override
+          public void run(Timeout timeout) throws Exception {
+            if (EnvironmentEdgeManager.currentTime() < endTime) {
+              addListener(isSnapshotFinished(snapshotDesc), (done, err2) -> {
+                if (err2 != null) {
+                  future.completeExceptionally(err2);
+                } else if (done) {
+                  future.complete(null);
+                } else {
+                  // retry again after pauseTime.
+                  long pauseTime =
+                    ConnectionUtils.getPauseTime(TimeUnit.NANOSECONDS.toMillis(pauseNs), ++tries);
                   pauseTime = Math.min(pauseTime, maxPauseTime);
-                  AsyncConnectionImpl.RETRY_TIMER
-                      .newTimeout(this, pauseTime, TimeUnit.MILLISECONDS);
+                  AsyncConnectionImpl.RETRY_TIMER.newTimeout(this, pauseTime,
+                    TimeUnit.MILLISECONDS);
                 }
-              } );
-              } else {
-                future.completeExceptionally(new SnapshotCreationException("Snapshot '"
-                    + snapshot.getName() + "' wasn't completed in expectedTime:" + expectedTimeout
-                    + " ms", snapshotDesc));
-              }
+              });
+            } else {
+              future.completeExceptionally(
+                new SnapshotCreationException("Snapshot '" + snapshot.getName() +
+                  "' wasn't completed in expectedTime:" + expectedTimeout + " ms", snapshotDesc));
             }
-          };
-          AsyncConnectionImpl.RETRY_TIMER.newTimeout(pollingTask, 1, TimeUnit.MILLISECONDS);
-        });
+          }
+        };
+        AsyncConnectionImpl.RETRY_TIMER.newTimeout(pollingTask, 1, TimeUnit.MILLISECONDS);
+      });
     return future;
   }
 
@@ -1808,52 +1797,50 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<Void> restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot) {
+  public CompletableFuture<Void> restoreSnapshot(String snapshotName,
+      boolean takeFailSafeSnapshot) {
     CompletableFuture<Void> future = new CompletableFuture<>();
-    listSnapshots(Pattern.compile(snapshotName)).whenComplete(
-      (snapshotDescriptions, err) -> {
-        if (err != null) {
-          future.completeExceptionally(err);
-          return;
-        }
-        TableName tableName = null;
-        if (snapshotDescriptions != null && !snapshotDescriptions.isEmpty()) {
-          for (SnapshotDescription snap : snapshotDescriptions) {
-            if (snap.getName().equals(snapshotName)) {
-              tableName = snap.getTableName();
-              break;
-            }
+    addListener(listSnapshots(Pattern.compile(snapshotName)), (snapshotDescriptions, err) -> {
+      if (err != null) {
+        future.completeExceptionally(err);
+        return;
+      }
+      TableName tableName = null;
+      if (snapshotDescriptions != null && !snapshotDescriptions.isEmpty()) {
+        for (SnapshotDescription snap : snapshotDescriptions) {
+          if (snap.getName().equals(snapshotName)) {
+            tableName = snap.getTableName();
+            break;
           }
         }
-        if (tableName == null) {
-          future.completeExceptionally(new RestoreSnapshotException(
-              "Unable to find the table name for snapshot=" + snapshotName));
-          return;
-        }
-        final TableName finalTableName = tableName;
-        tableExists(finalTableName)
-            .whenComplete((exists, err2) -> {
-              if (err2 != null) {
-                future.completeExceptionally(err2);
-              } else if (!exists) {
-                // if table does not exist, then just clone snapshot into new table.
-              completeConditionalOnFuture(future,
-                internalRestoreSnapshot(snapshotName, finalTableName));
+      }
+      if (tableName == null) {
+        future.completeExceptionally(new RestoreSnapshotException(
+          "Unable to find the table name for snapshot=" + snapshotName));
+        return;
+      }
+      final TableName finalTableName = tableName;
+      addListener(tableExists(finalTableName), (exists, err2) -> {
+        if (err2 != null) {
+          future.completeExceptionally(err2);
+        } else if (!exists) {
+          // if table does not exist, then just clone snapshot into new table.
+          completeConditionalOnFuture(future,
+            internalRestoreSnapshot(snapshotName, finalTableName));
+        } else {
+          addListener(isTableDisabled(finalTableName), (disabled, err4) -> {
+            if (err4 != null) {
+              future.completeExceptionally(err4);
+            } else if (!disabled) {
+              future.completeExceptionally(new TableNotDisabledException(finalTableName));
             } else {
-              isTableDisabled(finalTableName).whenComplete(
-                (disabled, err4) -> {
-                  if (err4 != null) {
-                    future.completeExceptionally(err4);
-                  } else if (!disabled) {
-                    future.completeExceptionally(new TableNotDisabledException(finalTableName));
-                  } else {
-                    completeConditionalOnFuture(future,
-                      restoreSnapshot(snapshotName, finalTableName, takeFailSafeSnapshot));
-                  }
-                });
+              completeConditionalOnFuture(future,
+                restoreSnapshot(snapshotName, finalTableName, takeFailSafeSnapshot));
             }
-          } );
+          });
+        }
       });
+    });
     return future;
   }
 
@@ -1862,49 +1849,50 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
     if (takeFailSafeSnapshot) {
       CompletableFuture<Void> future = new CompletableFuture<>();
       // Step.1 Take a snapshot of the current state
-      String failSafeSnapshotSnapshotNameFormat = this.connection.getConfiguration().get(
-        HConstants.SNAPSHOT_RESTORE_FAILSAFE_NAME,
-        HConstants.DEFAULT_SNAPSHOT_RESTORE_FAILSAFE_NAME);
-      final String failSafeSnapshotSnapshotName = failSafeSnapshotSnapshotNameFormat
-          .replace("{snapshot.name}", snapshotName)
+      String failSafeSnapshotSnapshotNameFormat =
+        this.connection.getConfiguration().get(HConstants.SNAPSHOT_RESTORE_FAILSAFE_NAME,
+          HConstants.DEFAULT_SNAPSHOT_RESTORE_FAILSAFE_NAME);
+      final String failSafeSnapshotSnapshotName =
+        failSafeSnapshotSnapshotNameFormat.replace("{snapshot.name}", snapshotName)
           .replace("{table.name}", tableName.toString().replace(TableName.NAMESPACE_DELIM, '.'))
           .replace("{restore.timestamp}", String.valueOf(EnvironmentEdgeManager.currentTime()));
       LOG.info("Taking restore-failsafe snapshot: " + failSafeSnapshotSnapshotName);
-      snapshot(failSafeSnapshotSnapshotName, tableName).whenComplete((ret, err) -> {
+      addListener(snapshot(failSafeSnapshotSnapshotName, tableName), (ret, err) -> {
         if (err != null) {
           future.completeExceptionally(err);
         } else {
           // Step.2 Restore snapshot
-        internalRestoreSnapshot(snapshotName, tableName).whenComplete((void2, err2) -> {
-          if (err2 != null) {
-            // Step.3.a Something went wrong during the restore and try to rollback.
-          internalRestoreSnapshot(failSafeSnapshotSnapshotName, tableName).whenComplete(
-            (void3, err3) -> {
-              if (err3 != null) {
-                future.completeExceptionally(err3);
-              } else {
-                String msg = "Restore snapshot=" + snapshotName + " failed. Rollback to snapshot="
-                    + failSafeSnapshotSnapshotName + " succeeded.";
-                future.completeExceptionally(new RestoreSnapshotException(msg));
-              }
-            });
-        } else {
-          // Step.3.b If the restore is succeeded, delete the pre-restore snapshot.
-          LOG.info("Deleting restore-failsafe snapshot: " + failSafeSnapshotSnapshotName);
-          deleteSnapshot(failSafeSnapshotSnapshotName).whenComplete(
-            (ret3, err3) -> {
-              if (err3 != null) {
-                LOG.error(
-                  "Unable to remove the failsafe snapshot: " + failSafeSnapshotSnapshotName, err3);
-                future.completeExceptionally(err3);
-              } else {
-                future.complete(ret3);
-              }
-            });
+          addListener(internalRestoreSnapshot(snapshotName, tableName), (void2, err2) -> {
+            if (err2 != null) {
+              // Step.3.a Something went wrong during the restore and try to rollback.
+              addListener(internalRestoreSnapshot(failSafeSnapshotSnapshotName, tableName),
+                (void3, err3) -> {
+                  if (err3 != null) {
+                    future.completeExceptionally(err3);
+                  } else {
+                    String msg =
+                      "Restore snapshot=" + snapshotName + " failed. Rollback to snapshot=" +
+                        failSafeSnapshotSnapshotName + " succeeded.";
+                    future.completeExceptionally(new RestoreSnapshotException(msg));
+                  }
+                });
+            } else {
+              // Step.3.b If the restore is succeeded, delete the pre-restore snapshot.
+              LOG.info("Deleting restore-failsafe snapshot: " + failSafeSnapshotSnapshotName);
+              addListener(deleteSnapshot(failSafeSnapshotSnapshotName), (ret3, err3) -> {
+                if (err3 != null) {
+                  LOG.error(
+                    "Unable to remove the failsafe snapshot: " + failSafeSnapshotSnapshotName,
+                    err3);
+                  future.completeExceptionally(err3);
+                } else {
+                  future.complete(ret3);
+                }
+              });
+            }
+          });
         }
-      } );
-      }
-    } );
+      });
       return future;
     } else {
       return internalRestoreSnapshot(snapshotName, tableName);
@@ -1913,7 +1901,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
 
   private <T> void completeConditionalOnFuture(CompletableFuture<T> dependentFuture,
       CompletableFuture<T> parentFuture) {
-    parentFuture.whenComplete((res, err) -> {
+    addListener(parentFuture, (res, err) -> {
       if (err != null) {
         dependentFuture.completeExceptionally(err);
       } else {
@@ -1925,7 +1913,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   @Override
   public CompletableFuture<Void> cloneSnapshot(String snapshotName, TableName tableName) {
     CompletableFuture<Void> future = new CompletableFuture<>();
-    tableExists(tableName).whenComplete((exists, err) -> {
+    addListener(tableExists(tableName), (exists, err) -> {
       if (err != null) {
         future.completeExceptionally(err);
       } else if (exists) {
@@ -1995,31 +1983,29 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   private CompletableFuture<List<SnapshotDescription>> getCompletedSnapshots(
       Pattern tableNamePattern, Pattern snapshotNamePattern) {
     CompletableFuture<List<SnapshotDescription>> future = new CompletableFuture<>();
-    listTableNames(tableNamePattern, false).whenComplete(
-      (tableNames, err) -> {
-        if (err != null) {
-          future.completeExceptionally(err);
+    addListener(listTableNames(tableNamePattern, false), (tableNames, err) -> {
+      if (err != null) {
+        future.completeExceptionally(err);
+        return;
+      }
+      if (tableNames == null || tableNames.size() <= 0) {
+        future.complete(Collections.emptyList());
+        return;
+      }
+      addListener(getCompletedSnapshots(snapshotNamePattern), (snapshotDescList, err2) -> {
+        if (err2 != null) {
+          future.completeExceptionally(err2);
           return;
         }
-        if (tableNames == null || tableNames.size() <= 0) {
+        if (snapshotDescList == null || snapshotDescList.isEmpty()) {
           future.complete(Collections.emptyList());
           return;
         }
-        getCompletedSnapshots(snapshotNamePattern).whenComplete(
-          (snapshotDescList, err2) -> {
-            if (err2 != null) {
-              future.completeExceptionally(err2);
-              return;
-            }
-            if (snapshotDescList == null || snapshotDescList.isEmpty()) {
-              future.complete(Collections.emptyList());
-              return;
-            }
-            future.complete(snapshotDescList.stream()
-                .filter(snap -> (snap != null && tableNames.contains(snap.getTableName())))
-                .collect(Collectors.toList()));
-          });
+        future.complete(snapshotDescList.stream()
+          .filter(snap -> (snap != null && tableNames.contains(snap.getTableName())))
+          .collect(Collectors.toList()));
       });
+    });
     return future;
   }
 
@@ -2066,7 +2052,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
       listSnapshotsFuture = getCompletedSnapshots(tableNamePattern, snapshotNamePattern);
     }
     CompletableFuture<Void> future = new CompletableFuture<>();
-    listSnapshotsFuture.whenComplete(((snapshotDescriptions, err) -> {
+    addListener(listSnapshotsFuture, ((snapshotDescriptions, err) -> {
       if (err != null) {
         future.completeExceptionally(err);
         return;
@@ -2075,12 +2061,14 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
         future.complete(null);
         return;
       }
-      List<CompletableFuture<Void>> deleteSnapshotFutures = new ArrayList<>();
-      snapshotDescriptions.forEach(snapDesc -> deleteSnapshotFutures
-          .add(internalDeleteSnapshot(snapDesc)));
-      CompletableFuture.allOf(
-        deleteSnapshotFutures.toArray(new CompletableFuture<?>[deleteSnapshotFutures.size()]))
-          .thenAccept(v -> future.complete(v));
+      addListener(CompletableFuture.allOf(snapshotDescriptions.stream()
+        .map(this::internalDeleteSnapshot).toArray(CompletableFuture[]::new)), (v, e) -> {
+          if (e != null) {
+            future.completeExceptionally(e);
+          } else {
+            future.complete(v);
+          }
+        });
     }));
     return future;
   }
@@ -2102,50 +2090,50 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
       Map<String, String> props) {
     CompletableFuture<Void> future = new CompletableFuture<>();
     ProcedureDescription procDesc =
-        ProtobufUtil.buildProcedureDescription(signature, instance, props);
-    this.<Long> newMasterCaller()
-        .action((controller, stub) -> this.<ExecProcedureRequest, ExecProcedureResponse, Long> call(
-          controller, stub, ExecProcedureRequest.newBuilder().setProcedure(procDesc).build(),
-          (s, c, req, done) -> s.execProcedure(c, req, done), resp -> resp.getExpectedTimeout()))
-        .call().whenComplete((expectedTimeout, err) -> {
-          if (err != null) {
-            future.completeExceptionally(err);
-            return;
-          }
-          TimerTask pollingTask = new TimerTask() {
-            int tries = 0;
-            long startTime = EnvironmentEdgeManager.currentTime();
-            long endTime = startTime + expectedTimeout;
-            long maxPauseTime = expectedTimeout / maxAttempts;
-
-            @Override
-            public void run(Timeout timeout) throws Exception {
-              if (EnvironmentEdgeManager.currentTime() < endTime) {
-                isProcedureFinished(signature, instance, props).whenComplete((done, err2) -> {
-                  if (err2 != null) {
-                    future.completeExceptionally(err2);
-                    return;
-                  }
-                  if (done) {
-                    future.complete(null);
-                  } else {
-                    // retry again after pauseTime.
-                    long pauseTime = ConnectionUtils
-                        .getPauseTime(TimeUnit.NANOSECONDS.toMillis(pauseNs), ++tries);
-                    pauseTime = Math.min(pauseTime, maxPauseTime);
-                    AsyncConnectionImpl.RETRY_TIMER.newTimeout(this, pauseTime,
-                      TimeUnit.MICROSECONDS);
-                  }
-                });
-              } else {
-                future.completeExceptionally(new IOException("Procedure '" + signature + " : "
-                    + instance + "' wasn't completed in expectedTime:" + expectedTimeout + " ms"));
-              }
+      ProtobufUtil.buildProcedureDescription(signature, instance, props);
+    addListener(this.<Long> newMasterCaller()
+      .action((controller, stub) -> this.<ExecProcedureRequest, ExecProcedureResponse, Long> call(
+        controller, stub, ExecProcedureRequest.newBuilder().setProcedure(procDesc).build(),
+        (s, c, req, done) -> s.execProcedure(c, req, done), resp -> resp.getExpectedTimeout()))
+      .call(), (expectedTimeout, err) -> {
+        if (err != null) {
+          future.completeExceptionally(err);
+          return;
+        }
+        TimerTask pollingTask = new TimerTask() {
+          int tries = 0;
+          long startTime = EnvironmentEdgeManager.currentTime();
+          long endTime = startTime + expectedTimeout;
+          long maxPauseTime = expectedTimeout / maxAttempts;
+
+          @Override
+          public void run(Timeout timeout) throws Exception {
+            if (EnvironmentEdgeManager.currentTime() < endTime) {
+              addListener(isProcedureFinished(signature, instance, props), (done, err2) -> {
+                if (err2 != null) {
+                  future.completeExceptionally(err2);
+                  return;
+                }
+                if (done) {
+                  future.complete(null);
+                } else {
+                  // retry again after pauseTime.
+                  long pauseTime =
+                    ConnectionUtils.getPauseTime(TimeUnit.NANOSECONDS.toMillis(pauseNs), ++tries);
+                  pauseTime = Math.min(pauseTime, maxPauseTime);
+                  AsyncConnectionImpl.RETRY_TIMER.newTimeout(this, pauseTime,
+                    TimeUnit.MICROSECONDS);
+                }
+              });
+            } else {
+              future.completeExceptionally(new IOException("Procedure '" + signature + " : " +
+                instance + "' wasn't completed in expectedTime:" + expectedTimeout + " ms"));
             }
-          };
-          // Queue the polling task into RETRY_TIMER to poll procedure state asynchronously.
-          AsyncConnectionImpl.RETRY_TIMER.newTimeout(pollingTask, 1, TimeUnit.MILLISECONDS);
-        });
+          }
+        };
+        // Queue the polling task into RETRY_TIMER to poll procedure state asynchronously.
+        AsyncConnectionImpl.RETRY_TIMER.newTimeout(pollingTask, 1, TimeUnit.MILLISECONDS);
+      });
     return future;
   }
 
@@ -2264,15 +2252,15 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
       }
 
       CompletableFuture<HRegionLocation> returnedFuture = new CompletableFuture<>();
-      future.whenComplete((location, err) -> {
+      addListener(future, (location, err) -> {
         if (err != null) {
           returnedFuture.completeExceptionally(err);
           return;
         }
         if (!location.isPresent() || location.get().getRegion() == null) {
-          returnedFuture.completeExceptionally(new UnknownRegionException(
-              "Invalid region name or encoded region name: "
-                  + Bytes.toStringBinary(regionNameOrEncodedRegionName)));
+          returnedFuture.completeExceptionally(
+            new UnknownRegionException("Invalid region name or encoded region name: " +
+              Bytes.toStringBinary(regionNameOrEncodedRegionName)));
         } else {
           returnedFuture.complete(location.get());
         }
@@ -2296,14 +2284,14 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
     }
 
     if (Bytes.equals(regionNameOrEncodedRegionName,
-      RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionName())
-        || Bytes.equals(regionNameOrEncodedRegionName,
+      RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionName()) ||
+      Bytes.equals(regionNameOrEncodedRegionName,
         RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedNameAsBytes())) {
       return CompletableFuture.completedFuture(RegionInfoBuilder.FIRST_META_REGIONINFO);
     }
 
     CompletableFuture<RegionInfo> future = new CompletableFuture<>();
-    getRegionLocation(regionNameOrEncodedRegionName).whenComplete((location, err) -> {
+    addListener(getRegionLocation(regionNameOrEncodedRegionName), (location, err) -> {
       if (err != null) {
         future.completeExceptionally(err);
       } else {
@@ -2345,7 +2333,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
     }
   }
 
-  private abstract class ProcedureBiConsumer implements BiConsumer<Void, Throwable> {
+  private static abstract class ProcedureBiConsumer implements BiConsumer<Void, Throwable> {
 
     abstract void onFinished();
 
@@ -2361,7 +2349,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
     }
   }
 
-  private abstract class TableProcedureBiConsumer extends ProcedureBiConsumer {
+  private static abstract class TableProcedureBiConsumer extends ProcedureBiConsumer {
     protected final TableName tableName;
 
     TableProcedureBiConsumer(TableName tableName) {
@@ -2386,7 +2374,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
     }
   }
 
-  private abstract class NamespaceProcedureBiConsumer extends ProcedureBiConsumer {
+  private static abstract class NamespaceProcedureBiConsumer extends ProcedureBiConsumer {
     protected final String namespaceName;
 
     NamespaceProcedureBiConsumer(String namespaceName) {
@@ -2410,7 +2398,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
     }
   }
 
-  private class CreateTableProcedureBiConsumer extends TableProcedureBiConsumer {
+  private static class CreateTableProcedureBiConsumer extends TableProcedureBiConsumer {
 
     CreateTableProcedureBiConsumer(TableName tableName) {
       super(tableName);
@@ -2422,7 +2410,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
     }
   }
 
-  private class ModifyTableProcedureBiConsumer extends TableProcedureBiConsumer {
+  private static class ModifyTableProcedureBiConsumer extends TableProcedureBiConsumer {
 
     ModifyTableProcedureBiConsumer(AsyncAdmin admin, TableName tableName) {
       super(tableName);
@@ -2452,7 +2440,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
     }
   }
 
-  private class TruncateTableProcedureBiConsumer extends TableProcedureBiConsumer {
+  private static class TruncateTableProcedureBiConsumer extends TableProcedureBiConsumer {
 
     TruncateTableProcedureBiConsumer(TableName tableName) {
       super(tableName);
@@ -2464,7 +2452,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
     }
   }
 
-  private class EnableTableProcedureBiConsumer extends TableProcedureBiConsumer {
+  private static class EnableTableProcedureBiConsumer extends TableProcedureBiConsumer {
 
     EnableTableProcedureBiConsumer(TableName tableName) {
       super(tableName);
@@ -2476,7 +2464,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
     }
   }
 
-  private class DisableTableProcedureBiConsumer extends TableProcedureBiConsumer {
+  private static class DisableTableProcedureBiConsumer extends TableProcedureBiConsumer {
 
     DisableTableProcedureBiConsumer(TableName tableName) {
       super(tableName);
@@ -2488,7 +2476,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
     }
   }
 
-  private class AddColumnFamilyProcedureBiConsumer extends TableProcedureBiConsumer {
+  private static class AddColumnFamilyProcedureBiConsumer extends TableProcedureBiConsumer {
 
     AddColumnFamilyProcedureBiConsumer(TableName tableName) {
       super(tableName);
@@ -2500,7 +2488,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
     }
   }
 
-  private class DeleteColumnFamilyProcedureBiConsumer extends TableProcedureBiConsumer {
+  private static class DeleteColumnFamilyProcedureBiConsumer extends TableProcedureBiConsumer {
 
     DeleteColumnFamilyProcedureBiConsumer(TableName tableName) {
       super(tableName);
@@ -2512,7 +2500,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
     }
   }
 
-  private class ModifyColumnFamilyProcedureBiConsumer extends TableProcedureBiConsumer {
+  private static class ModifyColumnFamilyProcedureBiConsumer extends TableProcedureBiConsumer {
 
     ModifyColumnFamilyProcedureBiConsumer(TableName tableName) {
       super(tableName);
@@ -2524,7 +2512,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
     }
   }
 
-  private class CreateNamespaceProcedureBiConsumer extends NamespaceProcedureBiConsumer {
+  private static class CreateNamespaceProcedureBiConsumer extends NamespaceProcedureBiConsumer {
 
     CreateNamespaceProcedureBiConsumer(String namespaceName) {
       super(namespaceName);
@@ -2536,7 +2524,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
     }
   }
 
-  private class DeleteNamespaceProcedureBiConsumer extends NamespaceProcedureBiConsumer {
+  private static class DeleteNamespaceProcedureBiConsumer extends NamespaceProcedureBiConsumer {
 
     DeleteNamespaceProcedureBiConsumer(String namespaceName) {
       super(namespaceName);
@@ -2548,7 +2536,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
     }
   }
 
-  private class ModifyNamespaceProcedureBiConsumer extends NamespaceProcedureBiConsumer {
+  private static class ModifyNamespaceProcedureBiConsumer extends NamespaceProcedureBiConsumer {
 
     ModifyNamespaceProcedureBiConsumer(String namespaceName) {
       super(namespaceName);
@@ -2560,7 +2548,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
     }
   }
 
-  private class MergeTableRegionProcedureBiConsumer extends TableProcedureBiConsumer {
+  private static class MergeTableRegionProcedureBiConsumer extends TableProcedureBiConsumer {
 
     MergeTableRegionProcedureBiConsumer(TableName tableName) {
       super(tableName);
@@ -2572,7 +2560,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
     }
   }
 
-  private class SplitTableRegionProcedureBiConsumer extends  TableProcedureBiConsumer {
+  private static class SplitTableRegionProcedureBiConsumer extends TableProcedureBiConsumer {
 
     SplitTableRegionProcedureBiConsumer(TableName tableName) {
       super(tableName);
@@ -2584,7 +2572,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
     }
   }
 
-  private class ReplicationProcedureBiConsumer extends ProcedureBiConsumer {
+  private static class ReplicationProcedureBiConsumer extends ProcedureBiConsumer {
     private final String peerId;
     private final Supplier<String> getOperation;
 
@@ -2610,7 +2598,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
 
   private CompletableFuture<Void> waitProcedureResult(CompletableFuture<Long> procFuture) {
     CompletableFuture<Void> future = new CompletableFuture<>();
-    procFuture.whenComplete((procId, error) -> {
+    addListener(procFuture, (procId, error) -> {
       if (error != null) {
         future.completeExceptionally(error);
         return;
@@ -2621,30 +2609,33 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   }
 
   private void getProcedureResult(long procId, CompletableFuture<Void> future, int retries) {
-    this.<GetProcedureResultResponse> newMasterCaller().action((controller, stub) -> this
-        .<GetProcedureResultRequest, GetProcedureResultResponse, GetProcedureResultResponse> call(
-          controller, stub, GetProcedureResultRequest.newBuilder().setProcId(procId).build(),
-          (s, c, req, done) -> s.getProcedureResult(c, req, done), (resp) -> resp))
-        .call().whenComplete((response, error) -> {
-          if (error != null) {
-            LOG.warn("failed to get the procedure result procId={}", procId,
-              ConnectionUtils.translateException(error));
-            retryTimer.newTimeout(t -> getProcedureResult(procId, future, retries + 1),
-              ConnectionUtils.getPauseTime(pauseNs, retries), TimeUnit.NANOSECONDS);
-            return;
-          }
-          if (response.getState() == GetProcedureResultResponse.State.RUNNING) {
-            retryTimer.newTimeout(t -> getProcedureResult(procId, future, retries + 1),
-              ConnectionUtils.getPauseTime(pauseNs, retries), TimeUnit.NANOSECONDS);
-            return;
-          }
-          if (response.hasException()) {
-            IOException ioe = ForeignExceptionUtil.toIOException(response.getException());
-            future.completeExceptionally(ioe);
-          } else {
-            future.complete(null);
-          }
-        });
+    addListener(
+      this.<GetProcedureResultResponse> newMasterCaller()
+        .action((controller, stub) -> this
+          .<GetProcedureResultRequest, GetProcedureResultResponse, GetProcedureResultResponse> call(
+            controller, stub, GetProcedureResultRequest.newBuilder().setProcId(procId).build(),
+            (s, c, req, done) -> s.getProcedureResult(c, req, done), (resp) -> resp))
+        .call(),
+      (response, error) -> {
+        if (error != null) {
+          LOG.warn("failed to get the procedure result procId={}", procId,
+            ConnectionUtils.translateException(error));
+          retryTimer.newTimeout(t -> getProcedureResult(procId, future, retries + 1),
+            ConnectionUtils.getPauseTime(pauseNs, retries), TimeUnit.NANOSECONDS);
+          return;
+        }
+        if (response.getState() == GetProcedureResultResponse.State.RUNNING) {
+          retryTimer.newTimeout(t -> getProcedureResult(procId, future, retries + 1),
+            ConnectionUtils.getPauseTime(pauseNs, retries), TimeUnit.NANOSECONDS);
+          return;
+        }
+        if (response.hasException()) {
+          IOException ioe = ForeignExceptionUtil.toIOException(response.getException());
+          future.completeExceptionally(ioe);
+        } else {
+          future.complete(null);
+        }
+      });
   }
 
   private <T> CompletableFuture<T> failedFuture(Throwable error) {
@@ -2726,24 +2717,26 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   @Override
   public CompletableFuture<Void> updateConfiguration() {
     CompletableFuture<Void> future = new CompletableFuture<Void>();
-    getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS, Option.MASTER, Option.BACKUP_MASTERS))
-      .whenComplete((status, err) -> {
+    addListener(
+      getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS, Option.MASTER, Option.BACKUP_MASTERS)),
+      (status, err) -> {
         if (err != null) {
           future.completeExceptionally(err);
         } else {
           List<CompletableFuture<Void>> futures = new ArrayList<>();
           status.getLiveServerMetrics().keySet()
-              .forEach(server -> futures.add(updateConfiguration(server)));
+            .forEach(server -> futures.add(updateConfiguration(server)));
           futures.add(updateConfiguration(status.getMasterName()));
           status.getBackupMasterNames().forEach(master -> futures.add(updateConfiguration(master)));
-          CompletableFuture.allOf(futures.toArray(new CompletableFuture<?>[futures.size()]))
-              .whenComplete((result, err2) -> {
-                if (err2 != null) {
-                  future.completeExceptionally(err2);
-                } else {
-                  future.complete(result);
-                }
-              });
+          addListener(
+            CompletableFuture.allOf(futures.toArray(new CompletableFuture<?>[futures.size()])),
+            (result, err2) -> {
+              if (err2 != null) {
+                future.completeExceptionally(err2);
+              } else {
+                future.complete(result);
+              }
+            });
         }
       });
     return future;
@@ -2826,88 +2819,87 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
 
     switch (compactType) {
       case MOB:
-        connection.registry.getMasterAddress().whenComplete((serverName, err) -> {
+        addListener(connection.registry.getMasterAddress(), (serverName, err) -> {
           if (err != null) {
             future.completeExceptionally(err);
             return;
           }
           RegionInfo regionInfo = RegionInfo.createMobRegionInfo(tableName);
 
-          this.<GetRegionInfoResponse> newAdminCaller().serverName(serverName).action(
-            (controller, stub) -> this
-            .<GetRegionInfoRequest, GetRegionInfoResponse, GetRegionInfoResponse> adminCall(
+          addListener(this.<GetRegionInfoResponse> newAdminCaller().serverName(serverName)
+            .action((controller, stub) -> this
+              .<GetRegionInfoRequest, GetRegionInfoResponse, GetRegionInfoResponse> adminCall(
                 controller, stub,
                 RequestConverter.buildGetRegionInfoRequest(regionInfo.getRegionName(), true),
-                (s, c, req, done) -> s.getRegionInfo(controller, req, done), resp -> resp)
-          ).call().whenComplete((resp2, err2) -> {
-            if (err2 != null) {
-              future.completeExceptionally(err2);
-            } else {
-              if (resp2.hasCompactionState()) {
-                future.complete(ProtobufUtil.createCompactionState(resp2.getCompactionState()));
+                (s, c, req, done) -> s.getRegionInfo(controller, req, done), resp -> resp))
+            .call(), (resp2, err2) -> {
+              if (err2 != null) {
+                future.completeExceptionally(err2);
               } else {
-                future.complete(CompactionState.NONE);
+                if (resp2.hasCompactionState()) {
+                  future.complete(ProtobufUtil.createCompactionState(resp2.getCompactionState()));
+                } else {
+                  future.complete(CompactionState.NONE);
+                }
               }
-            }
-          });
+            });
         });
         break;
       case NORMAL:
-        getTableHRegionLocations(tableName).whenComplete(
-          (locations, err) -> {
-            if (err != null) {
-              future.completeExceptionally(err);
-              return;
-            }
-            List<CompactionState> regionStates = new ArrayList<>();
-            List<CompletableFuture<CompactionState>> futures = new ArrayList<>();
-            locations.stream().filter(loc -> loc.getServerName() != null)
-                .filter(loc -> loc.getRegion() != null)
-                .filter(loc -> !loc.getRegion().isOffline())
-                .map(loc -> loc.getRegion().getRegionName()).forEach(region -> {
-                  futures.add(getCompactionStateForRegion(region).whenComplete((regionState, err2) -> {
-                    // If any region compaction state is MAJOR_AND_MINOR
-                    // the table compaction state is MAJOR_AND_MINOR, too.
-                    if (err2 != null) {
-                      future.completeExceptionally(err2);
-                    } else if (regionState == CompactionState.MAJOR_AND_MINOR) {
-                      future.complete(regionState);
-                    } else {
-                      regionStates.add(regionState);
-                    }
-                  }));
-                });
-            CompletableFuture.allOf(futures.toArray(new CompletableFuture<?>[futures.size()]))
-                .whenComplete((ret, err3) -> {
-                  // If future not completed, check all regions's compaction state
-                  if (!future.isCompletedExceptionally() && !future.isDone()) {
-                    CompactionState state = CompactionState.NONE;
-                    for (CompactionState regionState : regionStates) {
-                      switch (regionState) {
-                        case MAJOR:
-                          if (state == CompactionState.MINOR) {
-                            future.complete(CompactionState.MAJOR_AND_MINOR);
-                          } else {
-                            state = CompactionState.MAJOR;
-                          }
-                          break;
-                        case MINOR:
-                          if (state == CompactionState.MAJOR) {
-                            future.complete(CompactionState.MAJOR_AND_MINOR);
-                          } else {
-                            state = CompactionState.MINOR;
-                          }
-                          break;
-                        case NONE:
-                        default:
+        addListener(getTableHRegionLocations(tableName), (locations, err) -> {
+          if (err != null) {
+            future.completeExceptionally(err);
+            return;
+          }
+          List<CompactionState> regionStates = new ArrayList<>();
+          List<CompletableFuture<CompactionState>> futures = new ArrayList<>();
+          locations.stream().filter(loc -> loc.getServerName() != null)
+            .filter(loc -> loc.getRegion() != null).filter(loc -> !loc.getRegion().isOffline())
+            .map(loc -> loc.getRegion().getRegionName()).forEach(region -> {
+              futures.add(getCompactionStateForRegion(region).whenComplete((regionState, err2) -> {
+                // If any region compaction state is MAJOR_AND_MINOR
+                // the table compaction state is MAJOR_AND_MINOR, too.
+                if (err2 != null) {
+                  future.completeExceptionally(err2);
+                } else if (regionState == CompactionState.MAJOR_AND_MINOR) {
+                  future.complete(regionState);
+                } else {
+                  regionStates.add(regionState);
+                }
+              }));
+            });
+          addListener(
+            CompletableFuture.allOf(futures.toArray(new CompletableFuture<?>[futures.size()])),
+            (ret, err3) -> {
+              // If future not completed, check all regions's compaction state
+              if (!future.isCompletedExceptionally() && !future.isDone()) {
+                CompactionState state = CompactionState.NONE;
+                for (CompactionState regionState : regionStates) {
+                  switch (regionState) {
+                    case MAJOR:
+                      if (state == CompactionState.MINOR) {
+                        future.complete(CompactionState.MAJOR_AND_MINOR);
+                      } else {
+                        state = CompactionState.MAJOR;
                       }
-                      if (!future.isDone()) {
-                        future.complete(state);
+                      break;
+                    case MINOR:
+                      if (state == CompactionState.MAJOR) {
+                        future.complete(CompactionState.MAJOR_AND_MINOR);
+                      } else {
+                        state = CompactionState.MINOR;
                       }
-                    }
+                      break;
+                    case NONE:
+                    default:
                   }
-                });
-          });
+                  if (!future.isDone()) {
+                    future.complete(state);
+                  }
+                }
+              }
+            });
+        });
         break;
       default:
         throw new IllegalArgumentException("Unknown compactType: " + compactType);
@@ -2919,37 +2911,38 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   @Override
   public CompletableFuture<CompactionState> getCompactionStateForRegion(byte[] regionName) {
     CompletableFuture<CompactionState> future = new CompletableFuture<>();
-    getRegionLocation(regionName).whenComplete(
-      (location, err) -> {
-        if (err != null) {
-          future.completeExceptionally(err);
-          return;
-        }
-        ServerName serverName = location.getServerName();
-        if (serverName == null) {
-          future.completeExceptionally(new NoServerForRegionException(Bytes
-              .toStringBinary(regionName)));
-          return;
-        }
+    addListener(getRegionLocation(regionName), (location, err) -> {
+      if (err != null) {
+        future.completeExceptionally(err);
+        return;
+      }
+      ServerName serverName = location.getServerName();
+      if (serverName == null) {
+        future
+          .completeExceptionally(new NoServerForRegionException(Bytes.toStringBinary(regionName)));
+        return;
+      }
+      addListener(
         this.<GetRegionInfoResponse> newAdminCaller()
-            .action(
-              (controller, stub) -> this
-                  .<GetRegionInfoRequest, GetRegionInfoResponse, GetRegionInfoResponse> adminCall(
-                    controller, stub, RequestConverter.buildGetRegionInfoRequest(location
-                        .getRegion().getRegionName(), true), (s, c, req, done) -> s
-                 

<TRUNCATED>

[15/15] hbase git commit: HBASE-21579 Use AsyncClusterConnection for HBaseInterClusterReplicationEndpoint

Posted by zh...@apache.org.
HBASE-21579 Use AsyncClusterConnection for HBaseInterClusterReplicationEndpoint


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

Branch: refs/heads/HBASE-21512
Commit: 04e6909adfdf2e3a5b92fadb13dd8fc06e15c5e3
Parents: 0d051ab
Author: zhangduo <zh...@apache.org>
Authored: Tue Jan 1 21:27:14 2019 +0800
Committer: Duo Zhang <zh...@apache.org>
Committed: Fri Jan 4 14:34:02 2019 +0800

----------------------------------------------------------------------
 .../hbase/client/AsyncRegionServerAdmin.java    | 14 +++++--
 .../hbase/protobuf/ReplicationProtbufUtil.java  | 35 +++++++++--------
 .../HBaseInterClusterReplicationEndpoint.java   | 31 +++++++--------
 .../regionserver/ReplicationSinkManager.java    | 40 +++++++-------------
 .../replication/SyncReplicationTestBase.java    | 12 +++---
 .../TestReplicationSinkManager.java             | 21 +++++-----
 6 files changed, 74 insertions(+), 79 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/04e6909a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionServerAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionServerAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionServerAdmin.java
index 9accd89..b9141a9 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionServerAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionServerAdmin.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.client;
 
 import java.io.IOException;
 import java.util.concurrent.CompletableFuture;
+import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.ipc.HBaseRpcController;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -94,9 +95,9 @@ public class AsyncRegionServerAdmin {
     void call(AdminService.Interface stub, HBaseRpcController controller, RpcCallback<RESP> done);
   }
 
-  private <RESP> CompletableFuture<RESP> call(RpcCall<RESP> rpcCall) {
+  private <RESP> CompletableFuture<RESP> call(RpcCall<RESP> rpcCall, CellScanner cellScanner) {
     CompletableFuture<RESP> future = new CompletableFuture<>();
-    HBaseRpcController controller = conn.rpcControllerFactory.newController();
+    HBaseRpcController controller = conn.rpcControllerFactory.newController(cellScanner);
     try {
       rpcCall.call(conn.getAdminStub(server), controller, new RpcCallback<RESP>() {
 
@@ -115,6 +116,10 @@ public class AsyncRegionServerAdmin {
     return future;
   }
 
+  private <RESP> CompletableFuture<RESP> call(RpcCall<RESP> rpcCall) {
+    return call(rpcCall, null);
+  }
+
   public CompletableFuture<GetRegionInfoResponse> getRegionInfo(GetRegionInfoRequest request) {
     return call((stub, controller, done) -> stub.getRegionInfo(controller, request, done));
   }
@@ -154,8 +159,9 @@ public class AsyncRegionServerAdmin {
   }
 
   public CompletableFuture<ReplicateWALEntryResponse> replicateWALEntry(
-      ReplicateWALEntryRequest request) {
-    return call((stub, controller, done) -> stub.replicateWALEntry(controller, request, done));
+      ReplicateWALEntryRequest request, CellScanner cellScanner) {
+    return call((stub, controller, done) -> stub.replicateWALEntry(controller, request, done),
+      cellScanner);
   }
 
   public CompletableFuture<ReplicateWALEntryResponse> replay(ReplicateWALEntryRequest request) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/04e6909a/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java
index c1b3911..74fad26 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java
@@ -20,51 +20,54 @@ package org.apache.hadoop.hbase.protobuf;
 
 
 import java.io.IOException;
+import java.io.InterruptedIOException;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
+import java.util.concurrent.ExecutionException;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.PrivateCellUtil;
+import org.apache.hadoop.hbase.client.AsyncRegionServerAdmin;
 import org.apache.hadoop.hbase.io.SizedCellScanner;
-import org.apache.hadoop.hbase.ipc.HBaseRpcController;
-import org.apache.hadoop.hbase.ipc.HBaseRpcControllerImpl;
 import org.apache.hadoop.hbase.regionserver.wal.WALCellCodec;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
 import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.yetus.audience.InterfaceAudience;
 
+import org.apache.hbase.thirdparty.com.google.common.base.Throwables;
 import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
 
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
 
 @InterfaceAudience.Private
 public class ReplicationProtbufUtil {
+
   /**
-   * A helper to replicate a list of WAL entries using admin protocol.
-   * @param admin Admin service
+   * A helper to replicate a list of WAL entries using region server admin
+   * @param admin the region server admin
    * @param entries Array of WAL entries to be replicated
    * @param replicationClusterId Id which will uniquely identify source cluster FS client
    *          configurations in the replication configuration directory
    * @param sourceBaseNamespaceDir Path to source cluster base namespace directory
    * @param sourceHFileArchiveDir Path to the source cluster hfile archive directory
-   * @throws java.io.IOException
    */
-  public static void replicateWALEntry(final AdminService.BlockingInterface admin,
-      final Entry[] entries, String replicationClusterId, Path sourceBaseNamespaceDir,
-      Path sourceHFileArchiveDir) throws IOException {
-    Pair<AdminProtos.ReplicateWALEntryRequest, CellScanner> p =
-        buildReplicateWALEntryRequest(entries, null, replicationClusterId, sourceBaseNamespaceDir,
-          sourceHFileArchiveDir);
-    HBaseRpcController controller = new HBaseRpcControllerImpl(p.getSecond());
+  public static void replicateWALEntry(AsyncRegionServerAdmin admin, Entry[] entries,
+      String replicationClusterId, Path sourceBaseNamespaceDir, Path sourceHFileArchiveDir)
+      throws IOException {
+    Pair<AdminProtos.ReplicateWALEntryRequest, CellScanner> p = buildReplicateWALEntryRequest(
+      entries, null, replicationClusterId, sourceBaseNamespaceDir, sourceHFileArchiveDir);
     try {
-      admin.replicateWALEntry(controller, p.getFirst());
-    } catch (org.apache.hbase.thirdparty.com.google.protobuf.ServiceException e) {
-      throw ProtobufUtil.getServiceException(e);
+      admin.replicateWALEntry(p.getFirst(), p.getSecond()).get();
+    } catch (InterruptedException e) {
+      throw (IOException) new InterruptedIOException().initCause(e);
+    } catch (ExecutionException e) {
+      Throwable cause = e.getCause();
+      Throwables.propagateIfPossible(cause, IOException.class);
+      throw new IOException(e);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/04e6909a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java
index 7db53aa..0359096 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java
@@ -39,7 +39,6 @@ import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
-
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
@@ -48,13 +47,16 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
+import org.apache.hadoop.hbase.client.AsyncRegionServerAdmin;
+import org.apache.hadoop.hbase.client.ClusterConnectionFactory;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.ipc.RpcServer;
 import org.apache.hadoop.hbase.protobuf.ReplicationProtbufUtil;
 import org.apache.hadoop.hbase.replication.HBaseReplicationEndpoint;
 import org.apache.hadoop.hbase.replication.regionserver.ReplicationSinkManager.SinkPeer;
+import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
@@ -65,8 +67,6 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface;
-
 /**
  * A {@link org.apache.hadoop.hbase.replication.ReplicationEndpoint}
  * implementation for replicating to another HBase cluster.
@@ -85,8 +85,7 @@ public class HBaseInterClusterReplicationEndpoint extends HBaseReplicationEndpoi
 
   private static final long DEFAULT_MAX_TERMINATION_WAIT_MULTIPLIER = 2;
 
-  private ClusterConnection conn;
-  private Configuration localConf;
+  private AsyncClusterConnection conn;
   private Configuration conf;
   // How long should we sleep for each retry
   private long sleepForRetries;
@@ -117,7 +116,6 @@ public class HBaseInterClusterReplicationEndpoint extends HBaseReplicationEndpoi
   public void init(Context context) throws IOException {
     super.init(context);
     this.conf = HBaseConfiguration.create(ctx.getConfiguration());
-    this.localConf = HBaseConfiguration.create(ctx.getLocalConfiguration());
     decorateConf();
     this.maxRetriesMultiplier = this.conf.getInt("replication.source.maxretriesmultiplier", 300);
     this.socketTimeoutMultiplier = this.conf.getInt("replication.source.socketTimeoutMultiplier",
@@ -132,12 +130,13 @@ public class HBaseInterClusterReplicationEndpoint extends HBaseReplicationEndpoi
     // TODO: This connection is replication specific or we should make it particular to
     // replication and make replication specific settings such as compression or codec to use
     // passing Cells.
-    this.conn = (ClusterConnection) ConnectionFactory.createConnection(this.conf);
+    this.conn =
+      ClusterConnectionFactory.createAsyncClusterConnection(conf, null, User.getCurrent());
     this.sleepForRetries =
         this.conf.getLong("replication.source.sleepforretries", 1000);
     this.metrics = context.getMetrics();
     // ReplicationQueueInfo parses the peerId out of the znode for us
-    this.replicationSinkMgr = new ReplicationSinkManager(conn, ctx.getPeerId(), this, this.conf);
+    this.replicationSinkMgr = new ReplicationSinkManager(conn, this, this.conf);
     // per sink thread pool
     this.maxThreads = this.conf.getInt(HConstants.REPLICATION_SOURCE_MAXTHREADS_KEY,
       HConstants.REPLICATION_SOURCE_MAXTHREADS_DEFAULT);
@@ -284,9 +283,10 @@ public class HBaseInterClusterReplicationEndpoint extends HBaseReplicationEndpoi
   }
 
   private void reconnectToPeerCluster() {
-    ClusterConnection connection = null;
+    AsyncClusterConnection connection = null;
     try {
-      connection = (ClusterConnection) ConnectionFactory.createConnection(this.conf);
+      connection =
+        ClusterConnectionFactory.createAsyncClusterConnection(conf, null, User.getCurrent());
     } catch (IOException ioe) {
       LOG.warn("Failed to create connection for peer cluster", ioe);
     }
@@ -367,7 +367,7 @@ public class HBaseInterClusterReplicationEndpoint extends HBaseReplicationEndpoi
         }
         continue;
       }
-      if (this.conn == null || this.conn.isClosed()) {
+      if (this.conn == null) {
         reconnectToPeerCluster();
       }
       try {
@@ -480,10 +480,11 @@ public class HBaseInterClusterReplicationEndpoint extends HBaseReplicationEndpoi
           entriesHashCode, entries.size(), size, replicationClusterId);
       }
       sinkPeer = replicationSinkMgr.getReplicationSink();
-      BlockingInterface rrs = sinkPeer.getRegionServer();
+      AsyncRegionServerAdmin rsAdmin = sinkPeer.getRegionServer();
       try {
-        ReplicationProtbufUtil.replicateWALEntry(rrs, entries.toArray(new Entry[entries.size()]),
-          replicationClusterId, baseNamespaceDir, hfileArchiveDir);
+        ReplicationProtbufUtil.replicateWALEntry(rsAdmin,
+          entries.toArray(new Entry[entries.size()]), replicationClusterId, baseNamespaceDir,
+          hfileArchiveDir);
         LOG.trace("Completed replicating batch {}", entriesHashCode);
       } catch (IOException e) {
         LOG.trace("Failed replicating batch {}", entriesHashCode, e);

http://git-wip-us.apache.org/repos/asf/hbase/blob/04e6909a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSinkManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSinkManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSinkManager.java
index 3cd7884..21b07ac 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSinkManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSinkManager.java
@@ -21,11 +21,11 @@ import java.io.IOException;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
-import java.util.Random;
+import java.util.concurrent.ThreadLocalRandom;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.client.ClusterConnection;
-import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
+import org.apache.hadoop.hbase.client.AsyncRegionServerAdmin;
 import org.apache.hadoop.hbase.replication.HBaseReplicationEndpoint;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
@@ -35,8 +35,6 @@ import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesti
 import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
 
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
-
 /**
  * Maintains a collection of peers to replicate to, and randomly selects a
  * single peer to replicate to per set of data to replicate. Also handles
@@ -61,9 +59,7 @@ public class ReplicationSinkManager {
   static final float DEFAULT_REPLICATION_SOURCE_RATIO = 0.5f;
 
 
-  private final Connection conn;
-
-  private final String peerClusterId;
+  private final AsyncClusterConnection conn;
 
   private final HBaseReplicationEndpoint endpoint;
 
@@ -77,8 +73,6 @@ public class ReplicationSinkManager {
   // replication sinks is refreshed
   private final int badSinkThreshold;
 
-  private final Random random;
-
   // A timestamp of the last time the list of replication peers changed
   private long lastUpdateToPeers;
 
@@ -88,26 +82,22 @@ public class ReplicationSinkManager {
   /**
    * Instantiate for a single replication peer cluster.
    * @param conn connection to the peer cluster
-   * @param peerClusterId identifier of the peer cluster
    * @param endpoint replication endpoint for inter cluster replication
    * @param conf HBase configuration, used for determining replication source ratio and bad peer
    *          threshold
    */
-  public ReplicationSinkManager(ClusterConnection conn, String peerClusterId,
-      HBaseReplicationEndpoint endpoint, Configuration conf) {
+  public ReplicationSinkManager(AsyncClusterConnection conn, HBaseReplicationEndpoint endpoint,
+      Configuration conf) {
     this.conn = conn;
-    this.peerClusterId = peerClusterId;
     this.endpoint = endpoint;
     this.badReportCounts = Maps.newHashMap();
     this.ratio = conf.getFloat("replication.source.ratio", DEFAULT_REPLICATION_SOURCE_RATIO);
-    this.badSinkThreshold = conf.getInt("replication.bad.sink.threshold",
-                                        DEFAULT_BAD_SINK_THRESHOLD);
-    this.random = new Random();
+    this.badSinkThreshold =
+      conf.getInt("replication.bad.sink.threshold", DEFAULT_BAD_SINK_THRESHOLD);
   }
 
   /**
    * Get a randomly-chosen replication sink to replicate to.
-   *
    * @return a replication sink to replicate to
    */
   public synchronized SinkPeer getReplicationSink() throws IOException {
@@ -119,8 +109,8 @@ public class ReplicationSinkManager {
     if (sinks.isEmpty()) {
       throw new IOException("No replication sinks are available");
     }
-    ServerName serverName = sinks.get(random.nextInt(sinks.size()));
-    return new SinkPeer(serverName, ((ClusterConnection) conn).getAdmin(serverName));
+    ServerName serverName = sinks.get(ThreadLocalRandom.current().nextInt(sinks.size()));
+    return new SinkPeer(serverName, conn.getRegionServerAdmin(serverName));
   }
 
   /**
@@ -160,7 +150,7 @@ public class ReplicationSinkManager {
    */
   public synchronized void chooseSinks() {
     List<ServerName> slaveAddresses = endpoint.getRegionServers();
-    Collections.shuffle(slaveAddresses, random);
+    Collections.shuffle(slaveAddresses, ThreadLocalRandom.current());
     int numSinks = (int) Math.ceil(slaveAddresses.size() * ratio);
     sinks = slaveAddresses.subList(0, numSinks);
     lastUpdateToPeers = System.currentTimeMillis();
@@ -182,9 +172,9 @@ public class ReplicationSinkManager {
    */
   public static class SinkPeer {
     private ServerName serverName;
-    private AdminService.BlockingInterface regionServer;
+    private AsyncRegionServerAdmin regionServer;
 
-    public SinkPeer(ServerName serverName, AdminService.BlockingInterface regionServer) {
+    public SinkPeer(ServerName serverName, AsyncRegionServerAdmin regionServer) {
       this.serverName = serverName;
       this.regionServer = regionServer;
     }
@@ -193,10 +183,8 @@ public class ReplicationSinkManager {
       return serverName;
     }
 
-    public AdminService.BlockingInterface getRegionServer() {
+    public AsyncRegionServerAdmin getRegionServer() {
       return regionServer;
     }
-
   }
-
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/04e6909a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SyncReplicationTestBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SyncReplicationTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SyncReplicationTestBase.java
index f373590..e0d112d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SyncReplicationTestBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SyncReplicationTestBase.java
@@ -36,7 +36,7 @@ import org.apache.hadoop.hbase.StartMiniClusterOption;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
 import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Put;
@@ -250,19 +250,19 @@ public class SyncReplicationTestBase {
   protected final void verifyReplicationRequestRejection(HBaseTestingUtility utility,
       boolean expectedRejection) throws Exception {
     HRegionServer regionServer = utility.getRSForFirstRegionInTable(TABLE_NAME);
-    ClusterConnection connection = regionServer.getClusterConnection();
+    AsyncClusterConnection connection = regionServer.getAsyncClusterConnection();
     Entry[] entries = new Entry[10];
     for (int i = 0; i < entries.length; i++) {
       entries[i] =
         new Entry(new WALKeyImpl(HConstants.EMPTY_BYTE_ARRAY, TABLE_NAME, 0), new WALEdit());
     }
     if (!expectedRejection) {
-      ReplicationProtbufUtil.replicateWALEntry(connection.getAdmin(regionServer.getServerName()),
-        entries, null, null, null);
+      ReplicationProtbufUtil.replicateWALEntry(
+        connection.getRegionServerAdmin(regionServer.getServerName()), entries, null, null, null);
     } else {
       try {
-        ReplicationProtbufUtil.replicateWALEntry(connection.getAdmin(regionServer.getServerName()),
-          entries, null, null, null);
+        ReplicationProtbufUtil.replicateWALEntry(
+          connection.getRegionServerAdmin(regionServer.getServerName()), entries, null, null, null);
         fail("Should throw IOException when sync-replication state is in A or DA");
       } catch (DoNotRetryIOException e) {
         assertTrue(e.getMessage().contains("Reject to apply to sink cluster"));

http://git-wip-us.apache.org/repos/asf/hbase/blob/04e6909a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSinkManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSinkManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSinkManager.java
index 39dabb4..60afd40 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSinkManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSinkManager.java
@@ -25,7 +25,8 @@ import java.util.List;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
+import org.apache.hadoop.hbase.client.AsyncRegionServerAdmin;
 import org.apache.hadoop.hbase.replication.HBaseReplicationEndpoint;
 import org.apache.hadoop.hbase.replication.regionserver.ReplicationSinkManager.SinkPeer;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
@@ -37,8 +38,6 @@ import org.junit.experimental.categories.Category;
 
 import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
-
 @Category({ReplicationTests.class, SmallTests.class})
 public class TestReplicationSinkManager {
 
@@ -46,16 +45,14 @@ public class TestReplicationSinkManager {
   public static final HBaseClassTestRule CLASS_RULE =
       HBaseClassTestRule.forClass(TestReplicationSinkManager.class);
 
-  private static final String PEER_CLUSTER_ID = "PEER_CLUSTER_ID";
-
   private HBaseReplicationEndpoint replicationEndpoint;
   private ReplicationSinkManager sinkManager;
 
   @Before
   public void setUp() {
     replicationEndpoint = mock(HBaseReplicationEndpoint.class);
-    sinkManager = new ReplicationSinkManager(mock(ClusterConnection.class),
-                      PEER_CLUSTER_ID, replicationEndpoint, new Configuration());
+    sinkManager = new ReplicationSinkManager(mock(AsyncClusterConnection.class),
+      replicationEndpoint, new Configuration());
   }
 
   @Test
@@ -100,7 +97,7 @@ public class TestReplicationSinkManager {
     // Sanity check
     assertEquals(1, sinkManager.getNumSinks());
 
-    SinkPeer sinkPeer = new SinkPeer(serverNameA, mock(AdminService.BlockingInterface.class));
+    SinkPeer sinkPeer = new SinkPeer(serverNameA, mock(AsyncRegionServerAdmin.class));
 
     sinkManager.reportBadSink(sinkPeer);
 
@@ -131,7 +128,7 @@ public class TestReplicationSinkManager {
 
     ServerName serverName = sinkManager.getSinksForTesting().get(0);
 
-    SinkPeer sinkPeer = new SinkPeer(serverName, mock(AdminService.BlockingInterface.class));
+    SinkPeer sinkPeer = new SinkPeer(serverName, mock(AsyncRegionServerAdmin.class));
 
     sinkManager.reportSinkSuccess(sinkPeer); // has no effect, counter does not go negative
     for (int i = 0; i <= ReplicationSinkManager.DEFAULT_BAD_SINK_THRESHOLD; i++) {
@@ -147,7 +144,7 @@ public class TestReplicationSinkManager {
     //
     serverName = sinkManager.getSinksForTesting().get(0);
 
-    sinkPeer = new SinkPeer(serverName, mock(AdminService.BlockingInterface.class));
+    sinkPeer = new SinkPeer(serverName, mock(AsyncRegionServerAdmin.class));
     for (int i = 0; i <= ReplicationSinkManager.DEFAULT_BAD_SINK_THRESHOLD-1; i++) {
       sinkManager.reportBadSink(sinkPeer);
     }
@@ -188,8 +185,8 @@ public class TestReplicationSinkManager {
     ServerName serverNameA = sinkList.get(0);
     ServerName serverNameB = sinkList.get(1);
 
-    SinkPeer sinkPeerA = new SinkPeer(serverNameA, mock(AdminService.BlockingInterface.class));
-    SinkPeer sinkPeerB = new SinkPeer(serverNameB, mock(AdminService.BlockingInterface.class));
+    SinkPeer sinkPeerA = new SinkPeer(serverNameA, mock(AsyncRegionServerAdmin.class));
+    SinkPeer sinkPeerB = new SinkPeer(serverNameB, mock(AsyncRegionServerAdmin.class));
 
     for (int i = 0; i <= ReplicationSinkManager.DEFAULT_BAD_SINK_THRESHOLD; i++) {
       sinkManager.reportBadSink(sinkPeerA);