You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by nd...@apache.org on 2020/01/30 01:18:15 UTC

[hbase] branch HBASE-18095/client-locate-meta-no-zookeeper updated (a5b63d9 -> 60cb4f3)

This is an automated email from the ASF dual-hosted git repository.

ndimiduk pushed a change to branch HBASE-18095/client-locate-meta-no-zookeeper
in repository https://gitbox.apache.org/repos/asf/hbase.git.


    omit a5b63d9  HBASE-23731: De-flake TestFromClientSide (#1091)
    omit 7b67538  HBASE-23647: Make MasterRegistry the default impl. (#1039)
    omit f1145f5  HBASE-23330: Fix delegation token fetch with MasterRegistry (#1084)
    omit 86259e4  HBASE-23305: Master based registry implementation (#954)
    omit 10d7d3b  HBASE-23648: Re-use underlying connection registry in RawAsyncHBaseAdmin (#994)
    omit cae000a  HBASE-23604: Clarify AsyncRegistry usage in the code. (#957)
    omit d5b37d0  HBASE-23304: RPCs needed for client meta information lookup (#904)
    omit 2792635  HBASE-23281: Track meta region locations in masters (#830)
    omit c2996c5  HBASE-23275: Track active master's address in ActiveMasterManager (#812)
     add 60b80b6  HBASE-23736 Removed deprecated getTimeStampOfLastAppliedOp from MetricsSink
     add 66d198d  HBASE-23749 : TestHFileWriterV3 for all DataBlock Encoders
     add 8b00f9f  HBASE-17115 Define UI admins via an ACL
     new 084c618  HBASE-23275: Track active master's address in ActiveMasterManager (#812)
     new f3aaba1  HBASE-23281: Track meta region locations in masters (#830)
     new f44f7e5  HBASE-23304: RPCs needed for client meta information lookup (#904)
     new 150edac  HBASE-23604: Clarify AsyncRegistry usage in the code. (#957)
     new 7cc800d  HBASE-23648: Re-use underlying connection registry in RawAsyncHBaseAdmin (#994)
     new cff8b77  HBASE-23305: Master based registry implementation (#954)
     new fcb2012  HBASE-23330: Fix delegation token fetch with MasterRegistry (#1084)
     new 89c0235  HBASE-23647: Make MasterRegistry the default impl. (#1039)
     new 60cb4f3  HBASE-23731: De-flake TestFromClientSide (#1091)

This update added new revisions after undoing existing revisions.
That is to say, some revisions that were in the old version of the
branch are not in the new version.  This situation occurs
when a user --force pushes a change and generates a repository
containing something like this:

 * -- * -- B -- O -- O -- O   (a5b63d9)
            \
             N -- N -- N   refs/heads/HBASE-18095/client-locate-meta-no-zookeeper (60cb4f3)

You should already have received notification emails for all of the O
revisions, and so the following emails describe only the N revisions
from the common base, B.

Any revisions marked "omit" are not gone; other references still
refer to them.  Any revisions marked "discard" are gone forever.

The 9 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 ...ntionFilter.java => AdminAuthorizedFilter.java} |  52 +--
 .../org/apache/hadoop/hbase/http/HttpServer.java   | 133 ++++---
 .../org/apache/hadoop/hbase/http/InfoServer.java   |  72 +++-
 .../org/apache/hadoop/hbase/http/log/LogLevel.java |   8 +
 .../apache/hadoop/hbase/http/TestHttpServer.java   |  12 +-
 .../hadoop/hbase/http/TestSSLHttpServer.java       |   2 +-
 .../hadoop/hbase/http/TestSpnegoHttpServer.java    |   4 +-
 .../org/apache/hadoop/hbase/master/HMaster.java    |   2 +-
 .../hadoop/hbase/regionserver/HRegionServer.java   |   4 +-
 .../replication/regionserver/MetricsSink.java      |  12 -
 .../resources/hbase-webapps/master/snapshot.jsp    |   3 +-
 .../main/resources/hbase-webapps/master/table.jsp  |   3 +-
 .../hadoop/hbase/http/TestInfoServersACL.java      | 438 +++++++++++++++++++++
 .../hadoop/hbase/io/hfile/TestHFileWriterV3.java   |  33 +-
 ...java => TestHFileWriterV3WithDataEncoders.java} | 306 +++++++-------
 src/main/asciidoc/_chapters/security.adoc          |  85 +++-
 16 files changed, 903 insertions(+), 266 deletions(-)
 copy hbase-http/src/main/java/org/apache/hadoop/hbase/http/{ClickjackingPreventionFilter.java => AdminAuthorizedFilter.java} (51%)
 create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestInfoServersACL.java
 copy hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/{TestHFileWriterV3.java => TestHFileWriterV3WithDataEncoders.java} (59%)


[hbase] 06/09: HBASE-23305: Master based registry implementation (#954)

Posted by nd...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

ndimiduk pushed a commit to branch HBASE-18095/client-locate-meta-no-zookeeper
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit cff8b77974ca61ad2803ba30dbf33ffb484d2b14
Author: Bharath Vissapragada <bh...@apache.org>
AuthorDate: Tue Jan 14 08:24:07 2020 -0800

    HBASE-23305: Master based registry implementation (#954)
    
    Implements a master based registry for clients.
    
     - Supports hedged RPCs (fan out configured via configs).
     - Parameterized existing client tests to run with multiple registry combinations.
     - Added unit-test coverage for the new registry implementation.
    
    Signed-off-by: Nick Dimiduk <nd...@apache.org>
    Signed-off-by: stack <st...@apache.org>
    Signed-off-by: Andrew Purtell <ap...@apache.org>
---
 .../hbase/client/ConnectionRegistryFactory.java    |   4 +-
 .../apache/hadoop/hbase/client/MasterRegistry.java | 226 +++++++++++
 .../MasterRegistryFetchException.java}             |  32 +-
 .../apache/hadoop/hbase/ipc/AbstractRpcClient.java |  54 +--
 .../apache/hadoop/hbase/ipc/BlockingRpcClient.java |   7 +-
 .../apache/hadoop/hbase/ipc/HedgedRpcChannel.java  | 274 ++++++++++++++
 .../apache/hadoop/hbase/ipc/NettyRpcClient.java    |  34 +-
 .../org/apache/hadoop/hbase/ipc/RpcClient.java     |  19 +-
 .../hbase/client/TestConnectionRegistryLeak.java   |   3 +-
 .../java/org/apache/hadoop/hbase/HConstants.java   |  20 +-
 .../apache/hadoop/hbase/util/PrettyPrinter.java    |  23 +-
 .../org/apache/hadoop/hbase/TableNameTestRule.java |  16 +-
 .../apache/hadoop/hbase/util/JVMClusterUtil.java   |   7 +
 .../apache/hadoop/hbase/HBaseTestingUtility.java   |   3 +
 .../hbase/client/DummyConnectionRegistry.java      |   3 +-
 .../hadoop/hbase/client/TestFromClientSide.java    | 420 +++++++++++----------
 .../client/TestFromClientSideWithCoprocessor.java  |  23 +-
 .../hadoop/hbase/client/TestMasterRegistry.java    | 125 ++++++
 .../hbase/client/TestScannersFromClientSide.java   | 136 ++++---
 .../apache/hadoop/hbase/ipc/AbstractTestIPC.java   | 120 +++++-
 .../hbase/ipc/TestProtobufRpcServiceImpl.java      |  25 +-
 21 files changed, 1241 insertions(+), 333 deletions(-)

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistryFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistryFactory.java
index 80d358b..9308443 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistryFactory.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistryFactory.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hbase.client;
 
+import static org.apache.hadoop.hbase.HConstants.CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.util.ReflectionUtils;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -27,9 +28,6 @@ import org.apache.yetus.audience.InterfaceAudience;
 @InterfaceAudience.Private
 final class ConnectionRegistryFactory {
 
-  static final String CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY =
-      "hbase.client.connection.registry.impl";
-
   private ConnectionRegistryFactory() {
   }
 
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
new file mode 100644
index 0000000..5680847
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
@@ -0,0 +1,226 @@
+/*
+ * 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.HConstants.MASTER_ADDRS_DEFAULT;
+import static org.apache.hadoop.hbase.HConstants.MASTER_ADDRS_KEY;
+import static org.apache.hadoop.hbase.HConstants.MASTER_REGISTRY_ENABLE_HEDGED_READS_DEFAULT;
+import static org.apache.hadoop.hbase.HConstants.MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Function;
+import java.util.function.Predicate;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.RegionLocations;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.exceptions.MasterRegistryFetchException;
+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.yetus.audience.InterfaceAudience;
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hbase.thirdparty.com.google.common.net.HostAndPort;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ClientMetaService;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetActiveMasterRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetActiveMasterResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterIdRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterIdResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetMetaRegionLocationsRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetMetaRegionLocationsResponse;
+
+/**
+ * Master based registry implementation. Makes RPCs to the configured master addresses from config
+ * {@value org.apache.hadoop.hbase.HConstants#MASTER_ADDRS_KEY}.
+ *
+ * It supports hedged reads, which can be enabled by setting
+ * {@value org.apache.hadoop.hbase.HConstants#MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY} to True. Fan
+ * out the requests batch is controlled by
+ * {@value org.apache.hadoop.hbase.HConstants#HBASE_RPCS_HEDGED_REQS_FANOUT_KEY}.
+ *
+ * TODO: Handle changes to the configuration dynamically without having to restart the client.
+ */
+@InterfaceAudience.Private
+public class MasterRegistry implements ConnectionRegistry {
+  private static final String MASTER_ADDRS_CONF_SEPARATOR = ",";
+
+  // Configured list of masters to probe the meta information from.
+  private final Set<ServerName> masterServers;
+
+  // RPC client used to talk to the masters.
+  private final RpcClient rpcClient;
+  private final RpcControllerFactory rpcControllerFactory;
+  private final int rpcTimeoutMs;
+
+  MasterRegistry(Configuration conf) {
+    boolean hedgedReadsEnabled = conf.getBoolean(MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY,
+        MASTER_REGISTRY_ENABLE_HEDGED_READS_DEFAULT);
+    Configuration finalConf;
+    if (!hedgedReadsEnabled) {
+      // If hedged reads are disabled, it is equivalent to setting a fan out of 1. We make a copy of
+      // the configuration so that other places reusing this reference is not affected.
+      finalConf = new Configuration(conf);
+      finalConf.setInt(HConstants.HBASE_RPCS_HEDGED_REQS_FANOUT_KEY, 1);
+    } else {
+      finalConf = conf;
+    }
+    rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE, conf.getLong(HConstants.HBASE_RPC_TIMEOUT_KEY,
+        HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
+    masterServers = new HashSet<>();
+    parseMasterAddrs(finalConf);
+    rpcClient = RpcClientFactory.createClient(finalConf, HConstants.CLUSTER_ID_DEFAULT);
+    rpcControllerFactory = RpcControllerFactory.instantiate(finalConf);
+  }
+
+  /**
+   * @return Stub needed to make RPC using a hedged channel to the master end points.
+   */
+  private ClientMetaService.Interface getMasterStub() throws IOException {
+    return ClientMetaService.newStub(
+        rpcClient.createHedgedRpcChannel(masterServers, User.getCurrent(), rpcTimeoutMs));
+  }
+
+  /**
+   * Parses the list of master addresses from the provided configuration. Supported format is
+   * comma separated host[:port] values. If no port number if specified, default master port is
+   * assumed.
+   * @param conf Configuration to parse from.
+   */
+  private void parseMasterAddrs(Configuration conf) {
+    String configuredMasters = conf.get(MASTER_ADDRS_KEY, MASTER_ADDRS_DEFAULT);
+    for (String masterAddr: configuredMasters.split(MASTER_ADDRS_CONF_SEPARATOR)) {
+      HostAndPort masterHostPort =
+          HostAndPort.fromString(masterAddr.trim()).withDefaultPort(HConstants.DEFAULT_MASTER_PORT);
+      masterServers.add(ServerName.valueOf(masterHostPort.toString(), ServerName.NON_STARTCODE));
+    }
+    Preconditions.checkArgument(!masterServers.isEmpty(), "At least one master address is needed");
+  }
+
+  @VisibleForTesting
+  public Set<ServerName> getParsedMasterServers() {
+    return Collections.unmodifiableSet(masterServers);
+  }
+
+  /**
+   * Returns a call back that can be passed along to the non-blocking rpc call. It is invoked once
+   * the rpc finishes and the response is propagated to the passed future.
+   * @param future Result future to which the rpc response is propagated.
+   * @param isValidResp Checks if the rpc response has a valid result.
+   * @param transformResult Transforms the result to a different form as expected by callers.
+   * @param hrc RpcController instance for this rpc.
+   * @param debug Debug message passed along to the caller in case of exceptions.
+   * @param <T> RPC result type.
+   * @param <R> Transformed type of the result.
+   * @return A call back that can be embedded in the non-blocking rpc call.
+   */
+  private <T, R> RpcCallback<T> getRpcCallBack(CompletableFuture<R> future,
+      Predicate<T> isValidResp, Function<T, R> transformResult, HBaseRpcController hrc,
+      final String debug) {
+    return rpcResult -> {
+      if (rpcResult == null) {
+        future.completeExceptionally(
+            new MasterRegistryFetchException(masterServers, hrc.getFailed()));
+      }
+      if (!isValidResp.test(rpcResult)) {
+        // Rpc returned ok, but result was malformed.
+        future.completeExceptionally(new IOException(
+            String.format("Invalid result for request %s. Will be retried", debug)));
+
+      }
+      future.complete(transformResult.apply(rpcResult));
+    };
+  }
+
+  /**
+   * Simple helper to transform the result of getMetaRegionLocations() rpc.
+   */
+  private RegionLocations transformMetaRegionLocations(GetMetaRegionLocationsResponse resp) {
+    List<HRegionLocation> regionLocations = new ArrayList<>();
+    resp.getMetaLocationsList().forEach(
+      location -> regionLocations.add(ProtobufUtil.toRegionLocation(location)));
+    return new RegionLocations(regionLocations);
+  }
+
+  @Override
+  public CompletableFuture<RegionLocations> getMetaRegionLocations() {
+    CompletableFuture<RegionLocations> result = new CompletableFuture<>();
+    HBaseRpcController hrc = rpcControllerFactory.newController();
+    RpcCallback<GetMetaRegionLocationsResponse> callback = getRpcCallBack(result,
+      (rpcResp) -> rpcResp.getMetaLocationsCount() != 0, this::transformMetaRegionLocations, hrc,
+        "getMetaRegionLocations()");
+    try {
+      getMasterStub().getMetaRegionLocations(
+          hrc, GetMetaRegionLocationsRequest.getDefaultInstance(), callback);
+    } catch (IOException e) {
+      result.completeExceptionally(e);
+    }
+    return result;
+  }
+
+  @Override
+  public CompletableFuture<String> getClusterId() {
+    CompletableFuture<String> result = new CompletableFuture<>();
+    HBaseRpcController hrc = rpcControllerFactory.newController();
+    RpcCallback<GetClusterIdResponse> callback = getRpcCallBack(result,
+        GetClusterIdResponse::hasClusterId,  GetClusterIdResponse::getClusterId, hrc,
+        "getClusterId()");
+    try {
+      getMasterStub().getClusterId(hrc, GetClusterIdRequest.getDefaultInstance(), callback);
+    } catch (IOException e) {
+      result.completeExceptionally(e);
+    }
+    return result;
+  }
+
+  private ServerName transformServerName(GetActiveMasterResponse resp) {
+    return ProtobufUtil.toServerName(resp.getServerName());
+  }
+
+  @Override
+  public CompletableFuture<ServerName> getActiveMaster() {
+    CompletableFuture<ServerName> result = new CompletableFuture<>();
+    HBaseRpcController hrc = rpcControllerFactory.newController();
+    RpcCallback<GetActiveMasterResponse> callback = getRpcCallBack(result,
+        GetActiveMasterResponse::hasServerName, this::transformServerName, hrc,
+        "getActiveMaster()");
+    try {
+      getMasterStub().getActiveMaster(hrc, GetActiveMasterRequest.getDefaultInstance(), callback);
+    } catch (IOException e) {
+      result.completeExceptionally(e);
+    }
+    return result;
+  }
+
+  @Override
+  public void close() {
+    if (rpcClient != null) {
+      rpcClient.close();
+    }
+  }
+}
\ No newline at end of file
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistryFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/MasterRegistryFetchException.java
similarity index 53%
copy from hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistryFactory.java
copy to hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/MasterRegistryFetchException.java
index 80d358b..18871be 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistryFactory.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/MasterRegistryFetchException.java
@@ -15,31 +15,23 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hbase.client;
+package org.apache.hadoop.hbase.exceptions;
 
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.util.ReflectionUtils;
+import java.util.Set;
+import org.apache.hadoop.hbase.HBaseIOException;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.util.PrettyPrinter;
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
- * Factory class to get the instance of configured connection registry.
+ * Exception thrown when an master registry RPC fails in client. The exception includes the list of
+ * masters to which RPC was attempted and the last exception encountered. Prior exceptions are
+ * included in the logs.
  */
 @InterfaceAudience.Private
-final class ConnectionRegistryFactory {
-
-  static final String CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY =
-      "hbase.client.connection.registry.impl";
-
-  private ConnectionRegistryFactory() {
-  }
-
-  /**
-   * @return The connection registry implementation to use.
-   */
-  static ConnectionRegistry getRegistry(Configuration conf) {
-    Class<? extends ConnectionRegistry> clazz = conf.getClass(
-        CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY, ZKConnectionRegistry.class,
-        ConnectionRegistry.class);
-    return ReflectionUtils.newInstance(clazz, conf);
+public class MasterRegistryFetchException extends HBaseIOException {
+  public MasterRegistryFetchException(Set<ServerName> masters, Throwable failure) {
+    super(String.format("Exception making rpc to masters %s", PrettyPrinter.toString(masters)),
+        failure);
   }
 }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
index d418720..b0baff2 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
@@ -20,39 +20,20 @@ package org.apache.hadoop.hbase.ipc;
 
 import static org.apache.hadoop.hbase.ipc.IPCUtil.toIOE;
 import static org.apache.hadoop.hbase.ipc.IPCUtil.wrapException;
-
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
-import org.apache.hbase.thirdparty.com.google.common.cache.CacheBuilder;
-import org.apache.hbase.thirdparty.com.google.common.cache.CacheLoader;
-import org.apache.hbase.thirdparty.com.google.common.cache.LoadingCache;
-import org.apache.hbase.thirdparty.com.google.protobuf.BlockingRpcChannel;
-import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors;
-import org.apache.hbase.thirdparty.com.google.protobuf.Message;
-import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
-import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
-import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
-import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
-
-import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer;
-
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.SocketAddress;
 import java.net.UnknownHostException;
 import java.util.Collection;
+import java.util.Set;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.ServerName;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hbase.client.MetricsConnection;
 import org.apache.hadoop.hbase.codec.Codec;
 import org.apache.hadoop.hbase.codec.KeyValueCodec;
@@ -63,7 +44,22 @@ import org.apache.hadoop.hbase.util.PoolMap;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.io.compress.CompressionCodec;
 import org.apache.hadoop.ipc.RemoteException;
-
+import org.apache.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.common.base.Preconditions;
+import org.apache.hbase.thirdparty.com.google.common.cache.CacheBuilder;
+import org.apache.hbase.thirdparty.com.google.common.cache.CacheLoader;
+import org.apache.hbase.thirdparty.com.google.common.cache.LoadingCache;
+import org.apache.hbase.thirdparty.com.google.protobuf.BlockingRpcChannel;
+import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
+import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
+import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
 
 /**
@@ -203,7 +199,9 @@ public abstract class AbstractRpcClient<T extends RpcConnection> implements RpcC
         // have some pending calls on connection so we should not shutdown the connection outside.
         // The connection itself will disconnect if there is no pending call for maxIdleTime.
         if (conn.getLastTouched() < closeBeforeTime && !conn.isActive()) {
-          if (LOG.isTraceEnabled()) LOG.trace("Cleanup idle connection to " + conn.remoteId().address);
+          if (LOG.isTraceEnabled()) {
+            LOG.trace("Cleanup idle connection to {}", conn.remoteId().address);
+          }
           connections.removeValue(conn.remoteId(), conn);
           conn.cleanupConnection();
         }
@@ -384,7 +382,7 @@ public abstract class AbstractRpcClient<T extends RpcConnection> implements RpcC
     }
   }
 
-  private void callMethod(final Descriptors.MethodDescriptor md, final HBaseRpcController hrc,
+  Call callMethod(final Descriptors.MethodDescriptor md, final HBaseRpcController hrc,
       final Message param, Message returnType, final User ticket, final InetSocketAddress addr,
       final RpcCallback<Message> callback) {
     final MetricsConnection.CallStats cs = MetricsConnection.newCallStats();
@@ -421,9 +419,10 @@ public abstract class AbstractRpcClient<T extends RpcConnection> implements RpcC
     } catch (Exception e) {
       call.setException(toIOE(e));
     }
+    return call;
   }
 
-  private InetSocketAddress createAddr(ServerName sn) throws UnknownHostException {
+  InetSocketAddress createAddr(ServerName sn) throws UnknownHostException {
     InetSocketAddress addr = new InetSocketAddress(sn.getHostname(), sn.getPort());
     if (addr.isUnresolved()) {
       throw new UnknownHostException("can not resolve " + sn.getServerName());
@@ -513,6 +512,13 @@ public abstract class AbstractRpcClient<T extends RpcConnection> implements RpcC
     return new RpcChannelImplementation(this, createAddr(sn), user, rpcTimeout);
   }
 
+  @Override
+  public RpcChannel createHedgedRpcChannel(Set<ServerName> sns, User user, int rpcTimeout)
+      throws UnknownHostException {
+    // Check HedgedRpcChannel implementation for detailed comments.
+    throw new UnsupportedOperationException("Hedging not supported for this implementation.");
+  }
+
   private static class AbstractRpcChannel {
 
     protected final InetSocketAddress addr;
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcClient.java
index f84c308..22eca53 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcClient.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcClient.java
@@ -17,18 +17,15 @@
  */
 package org.apache.hadoop.hbase.ipc;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
 import java.io.IOException;
 import java.net.SocketAddress;
-
 import javax.net.SocketFactory;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.client.MetricsConnection;
 import org.apache.hadoop.net.NetUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 
 /**
  * Does RPC against a cluster. Manages connections per regionserver in the cluster.
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HedgedRpcChannel.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HedgedRpcChannel.java
new file mode 100644
index 0000000..7b681e0
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HedgedRpcChannel.java
@@ -0,0 +1,274 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.ipc;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.PrettyPrinter;
+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.Descriptors;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
+
+/**
+ * A non-blocking implementation of RpcChannel that hedges requests to multiple service end points.
+ * First received response is returned to the caller. This abstracts out the logic needed to batch
+ * requests to multiple end points underneath and presents itself as a single logical RpcChannel to
+ * the client.
+ *
+ * Hedging Details:
+ * ---------------
+ * - Hedging of RPCs happens in multiple batches. In each iteration, we select a 'batch' of address
+ * end points to make the call to. We do multiple iterations until we get a proper response to the
+ * rpc call or all the service addresses are exhausted, which ever happens first. Size of each is
+ * configurable and is also known as 'fanOutSize'.
+ *
+ * - We randomize the addresses up front so that the batch order per client is non deterministic.
+ * This avoids hot spots on the service side. The size of each batch is controlled via 'fanOutSize'.
+ * Higher fanOutSize implies we make more rpc calls in a single batch. One needs to mindful of the
+ * load on the client and server side when configuring the fan out.
+ *
+ * - In a happy case, once we receive a response from one end point, we cancel all the
+ * other inflight rpcs in the same batch and return the response to the caller. If we do not get a
+ * valid response from any address end point, we propagate the error back to the caller.
+ *
+ * - Rpc timeouts are applied to every hedged rpc.
+ *
+ * - Callers need to be careful about what rpcs they are trying to hedge. Not every kind of call can
+ * be hedged (for example: cluster state changing rpcs).
+ *
+ * (TODO) Retries and Adaptive hedging policy:
+ * ------------------------------------------
+ *
+ * - No retries are handled at the channel level. Retries can be built in upper layers. However the
+ * question is, do we even need retries? Hedging in fact is a substitute for retries.
+ *
+ * - Clearly hedging puts more load on the service side. To mitigate this, we can make the hedging
+ * policy more adaptive. In most happy cases, the rpcs from the first few end points should return
+ * right away (especially short lived rpcs, that do not take up much time). In such cases, hedging
+ * is not needed. So, the idea is to make this request pattern pluggable so that the requests are
+ * hedged only when needed.
+ */
+class HedgedRpcChannel implements RpcChannel {
+  private static final Logger LOG = LoggerFactory.getLogger(HedgedRpcChannel.class);
+
+  /**
+   * Currently hedging is only supported for non-blocking connection implementation types because
+   * the channel implementation inherently relies on the connection implementation being async.
+   * Refer to the comments in doCallMethod().
+   */
+  private final NettyRpcClient rpcClient;
+  // List of service addresses to hedge the requests to.
+  private final List<InetSocketAddress> addrs;
+  private final User ticket;
+  private final int rpcTimeout;
+  // Controls the size of request fan out (number of rpcs per a single batch).
+  private final int fanOutSize;
+
+  /**
+   * A simple rpc call back implementation to notify the batch context if any rpc is successful.
+   */
+  private static class BatchRpcCtxCallBack implements RpcCallback<Message> {
+    private  final BatchRpcCtx batchRpcCtx;
+    private final HBaseRpcController rpcController;
+    BatchRpcCtxCallBack(BatchRpcCtx batchRpcCtx, HBaseRpcController rpcController) {
+      this.batchRpcCtx = batchRpcCtx;
+      this.rpcController = rpcController;
+    }
+    @Override
+    public void run(Message result) {
+      batchRpcCtx.setResultIfNotSet(result, rpcController);
+    }
+  }
+
+  /**
+   * A shared RPC context between a batch of hedged RPCs. Tracks the state and helpers needed to
+   * synchronize on multiple RPCs to different end points fetching the result. All the methods are
+   * thread-safe.
+   */
+  private static class BatchRpcCtx {
+    // Result set by the thread finishing first. Set only once.
+    private final AtomicReference<Message> result = new AtomicReference<>();
+    // Caller waits on this latch being set.
+    // We set this to 1, so that the first successful RPC result is returned to the client.
+    private CountDownLatch resultsReady = new CountDownLatch(1);
+    // Failed rpc book-keeping.
+    private AtomicInteger failedRpcCount = new AtomicInteger();
+    // All the call handles for this batch.
+    private final List<Call> callsInFlight = Collections.synchronizedList(new ArrayList<>());
+
+    // Target addresses.
+    private final List<InetSocketAddress> addresses;
+    // Called when the result is ready.
+    private final RpcCallback<Message> callBack;
+    // Last failed rpc's exception. Used to propagate the reason to the controller.
+    private IOException lastFailedRpcReason;
+
+
+    BatchRpcCtx(List<InetSocketAddress> addresses, RpcCallback<Message> callBack) {
+      this.addresses = addresses;
+      this.callBack = Preconditions.checkNotNull(callBack);
+    }
+
+    /**
+     * Sets the result only if it is not already set by another thread. Thread that successfully
+     * sets the result also count downs the latch.
+     * @param result Result to be set.
+     */
+    public void setResultIfNotSet(Message result, HBaseRpcController rpcController) {
+      if (rpcController.failed()) {
+        incrementFailedRpcs(rpcController.getFailed());
+        return;
+      }
+      if (this.result.compareAndSet(null, result)) {
+        resultsReady.countDown();
+        // Cancel all pending in flight calls.
+        for (Call call: callsInFlight) {
+          // It is ok to do it for all calls as it is a no-op if the call is already done.
+          final String exceptionMsg = String.format("%s canceled because another hedged attempt " +
+              "for the same rpc already succeeded. This is not needed anymore.", call);
+          call.setException(new CallCancelledException(exceptionMsg));
+        }
+      }
+    }
+
+    /**
+     * Waits until the results are populated and calls the callback if the call is successful.
+     * @return true for successful rpc and false otherwise.
+     */
+    public boolean waitForResults() {
+      try {
+        // We do not set a timeout on await() because we rely on the underlying RPCs to timeout if
+        // something on the remote is broken. Worst case we should wait for rpc time out to kick in.
+        resultsReady.await();
+      } catch (InterruptedException e) {
+        LOG.warn("Interrupted while waiting for batched master RPC results. Aborting wait.", e);
+      }
+      Message message = result.get();
+      if (message != null) {
+        callBack.run(message);
+        return true;
+      }
+      return false;
+    }
+
+    public void addCallInFlight(Call c) {
+      callsInFlight.add(c);
+    }
+
+    public void incrementFailedRpcs(IOException reason) {
+      if (failedRpcCount.incrementAndGet() == addresses.size()) {
+        lastFailedRpcReason = reason;
+        // All the rpcs in this batch have failed. Invoke the waiting threads.
+        resultsReady.countDown();
+      }
+    }
+
+    public IOException getLastFailedRpcReason() {
+      return lastFailedRpcReason;
+    }
+
+    @Override
+    public String toString() {
+      return String.format("Batched rpc for target(s) %s", PrettyPrinter.toString(addresses));
+    }
+  }
+
+  public HedgedRpcChannel(NettyRpcClient rpcClient, Set<InetSocketAddress> addrs,
+      User ticket, int rpcTimeout, int fanOutSize) {
+    this.rpcClient = rpcClient;
+    this.addrs = new ArrayList<>(Preconditions.checkNotNull(addrs));
+    Preconditions.checkArgument(this.addrs.size() >= 1);
+    // For non-deterministic client query pattern. Not all clients want to hedge RPCs in the same
+    // order, creating hot spots on the service end points.
+    Collections.shuffle(this.addrs);
+    this.ticket = ticket;
+    this.rpcTimeout = rpcTimeout;
+    // fanOutSize controls the number of hedged RPCs per batch.
+    this.fanOutSize = fanOutSize;
+  }
+
+  private HBaseRpcController applyRpcTimeout(RpcController controller) {
+    HBaseRpcController hBaseRpcController = (HBaseRpcController) controller;
+    int rpcTimeoutToSet =
+        hBaseRpcController.hasCallTimeout() ? hBaseRpcController.getCallTimeout() : rpcTimeout;
+    HBaseRpcController response = new HBaseRpcControllerImpl();
+    response.setCallTimeout(rpcTimeoutToSet);
+    return response;
+  }
+
+  private void doCallMethod(Descriptors.MethodDescriptor method, HBaseRpcController controller,
+      Message request, Message responsePrototype, RpcCallback<Message> done) {
+    int i = 0;
+    BatchRpcCtx lastBatchCtx = null;
+    while (i < addrs.size()) {
+      // Each iteration picks fanOutSize addresses to run as batch.
+      int batchEnd = Math.min(addrs.size(), i + fanOutSize);
+      List<InetSocketAddress> addrSubList = addrs.subList(i, batchEnd);
+      BatchRpcCtx batchRpcCtx = new BatchRpcCtx(addrSubList, done);
+      lastBatchCtx = batchRpcCtx;
+      LOG.debug("Attempting request {}, {}", method.getName(), batchRpcCtx);
+      for (InetSocketAddress address : addrSubList) {
+        HBaseRpcController rpcController = applyRpcTimeout(controller);
+        // ** WARN ** This is a blocking call if the underlying connection for the rpc client is
+        // a blocking implementation (ex: BlockingRpcConnection). That essentially serializes all
+        // the write calls. Handling blocking connection means that this should be run in a separate
+        // thread and hence more code complexity. Is it ok to handle only non-blocking connections?
+        batchRpcCtx.addCallInFlight(rpcClient.callMethod(method, rpcController, request,
+            responsePrototype, ticket, address,
+            new BatchRpcCtxCallBack(batchRpcCtx, rpcController)));
+      }
+      if (batchRpcCtx.waitForResults()) {
+        return;
+      }
+      // Entire batch has failed, lets try the next batch.
+      LOG.debug("Failed request {}, {}.", method.getName(), batchRpcCtx);
+      i = batchEnd;
+    }
+    Preconditions.checkNotNull(lastBatchCtx);
+    // All the batches failed, mark it a failed rpc.
+    // Propagate the failure reason. We propagate the last batch's last failing rpc reason.
+    // Can we do something better?
+    controller.setFailed(lastBatchCtx.getLastFailedRpcReason());
+    done.run(null);
+  }
+
+  @Override
+  public void callMethod(Descriptors.MethodDescriptor method, RpcController controller,
+      Message request, Message responsePrototype, RpcCallback<Message> done) {
+    // There is no reason to use any other implementation of RpcController.
+    Preconditions.checkState(controller instanceof HBaseRpcController);
+    // To make the channel non-blocking, we run the actual doCalMethod() async. The call back is
+    // called once the hedging finishes.
+    CompletableFuture.runAsync(
+      () -> doCallMethod(method, (HBaseRpcController)controller, request, responsePrototype, done));
+  }
+}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClient.java
index 61dedbb..c4f70b0 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClient.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClient.java
@@ -17,21 +17,26 @@
  */
 package org.apache.hadoop.hbase.ipc;
 
-import org.apache.hbase.thirdparty.io.netty.channel.Channel;
-import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup;
-import org.apache.hbase.thirdparty.io.netty.channel.nio.NioEventLoopGroup;
-import org.apache.hbase.thirdparty.io.netty.channel.socket.nio.NioSocketChannel;
-import org.apache.hbase.thirdparty.io.netty.util.concurrent.DefaultThreadFactory;
-
 import java.io.IOException;
+import java.net.InetSocketAddress;
 import java.net.SocketAddress;
-
+import java.net.UnknownHostException;
+import java.util.HashSet;
+import java.util.Set;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.client.MetricsConnection;
+import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.Pair;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
+import org.apache.hbase.thirdparty.io.netty.channel.Channel;
+import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup;
+import org.apache.hbase.thirdparty.io.netty.channel.nio.NioEventLoopGroup;
+import org.apache.hbase.thirdparty.io.netty.channel.socket.nio.NioSocketChannel;
+import org.apache.hbase.thirdparty.io.netty.util.concurrent.DefaultThreadFactory;
 
 /**
  * Netty client for the requests and responses.
@@ -75,6 +80,19 @@ public class NettyRpcClient extends AbstractRpcClient<NettyRpcConnection> {
   }
 
   @Override
+  public RpcChannel createHedgedRpcChannel(Set<ServerName> sns, User user, int rpcTimeout)
+      throws UnknownHostException {
+    final int hedgedRpcFanOut = conf.getInt(HConstants.HBASE_RPCS_HEDGED_REQS_FANOUT_KEY,
+        HConstants.HBASE_RPCS_HEDGED_REQS_FANOUT_DEFAULT);
+    Set<InetSocketAddress> addresses = new HashSet<>();
+    for (ServerName sn: sns) {
+      addresses.add(createAddr(sn));
+    }
+    return new HedgedRpcChannel(this, addresses, user, rpcTimeout,
+        hedgedRpcFanOut);
+  }
+
+  @Override
   protected void closeInternal() {
     if (shutdownGroupWhenClose) {
       group.shutdownGracefully();
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java
index 0e00695..558fcee 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java
@@ -17,15 +17,14 @@
  */
 package org.apache.hadoop.hbase.ipc;
 
-import org.apache.hbase.thirdparty.com.google.protobuf.BlockingRpcChannel;
-import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
-
 import java.io.Closeable;
 import java.io.IOException;
-
+import java.util.Set;
 import org.apache.hadoop.hbase.ServerName;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.security.User;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hbase.thirdparty.com.google.protobuf.BlockingRpcChannel;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
 
 /**
  * Interface for RpcClient implementations so ConnectionManager can handle it.
@@ -84,6 +83,16 @@ public interface RpcClient extends Closeable {
       throws IOException;
 
   /**
+   * Creates a channel that can hedge request to multiple underlying channels.
+   * @param sns  Set of servers for underlying channels.
+   * @param user user for the connection.
+   * @param rpcTimeout rpc timeout to use.
+   * @return A hedging rpc channel for this rpc client instance.
+   */
+  RpcChannel createHedgedRpcChannel(final Set<ServerName> sns, final User user, int rpcTimeout)
+       throws IOException;
+
+  /**
    * Interrupt the connections to the given server. This should be called if the server
    * is known as actually dead. This will not prevent current operation to be retried, and,
    * depending on their own behavior, they may retry on the same server. This can be a feature,
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestConnectionRegistryLeak.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestConnectionRegistryLeak.java
index f02ec42..561b1f5 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestConnectionRegistryLeak.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestConnectionRegistryLeak.java
@@ -29,6 +29,7 @@ import java.util.concurrent.ExecutionException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.FutureUtils;
@@ -70,7 +71,7 @@ public class TestConnectionRegistryLeak {
 
   @BeforeClass
   public static void setUp() {
-    CONF.setClass(ConnectionRegistryFactory.CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY,
+    CONF.setClass(HConstants.CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY,
         ConnectionRegistryForTest.class, ConnectionRegistry.class);
   }
 
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
index ce7abc8..a7acc8e 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
@@ -179,10 +179,17 @@ public final class HConstants {
   public static final String MASTER_INFO_PORT = "hbase.master.info.port";
 
   /** Configuration key for the list of master host:ports **/
-  public static final String MASTER_ADDRS_KEY = "hbase.master.addrs";
+  public static final String MASTER_ADDRS_KEY = "hbase.masters";
 
   public static final String MASTER_ADDRS_DEFAULT =  "localhost:" + DEFAULT_MASTER_PORT;
 
+  /** Configuration to enable hedged reads on master registry **/
+  public static final String MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY =
+      "hbase.client.master_registry.enable_hedged_reads";
+
+  /** Default value for enabling hedging reads on master registry **/
+  public static final boolean MASTER_REGISTRY_ENABLE_HEDGED_READS_DEFAULT = false;
+
   /** Parameter name for the master type being backup (waits for primary to go inactive). */
   public static final String MASTER_TYPE_BACKUP = "hbase.master.backup";
 
@@ -908,6 +915,12 @@ public final class HConstants {
    */
   public static final String HBASE_RPC_TIMEOUT_KEY = "hbase.rpc.timeout";
 
+  /** Configuration key that controls the fan out of requests in hedged channel implementation. **/
+  public static final String HBASE_RPCS_HEDGED_REQS_FANOUT_KEY = "hbase.rpc.hedged.fanout";
+
+  /** Default value for the fan out of hedged requests. **/
+  public static final int HBASE_RPCS_HEDGED_REQS_FANOUT_DEFAULT = 2;
+
   /**
    * timeout for each read RPC
    */
@@ -939,6 +952,11 @@ public final class HConstants {
    */
   public static final long NO_SEQNUM = -1;
 
+  /**
+   * Registry implementation to be used on the client side.
+   */
+  public static final String CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY =
+      "hbase.client.registry.impl";
 
   /*
    * cluster replication constants.
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/PrettyPrinter.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/PrettyPrinter.java
index 147e916..ff7064b 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/PrettyPrinter.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/PrettyPrinter.java
@@ -19,9 +19,12 @@
 
 package org.apache.hadoop.hbase.util;
 
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Objects;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
-
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.exceptions.HBaseException;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -29,7 +32,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @InterfaceAudience.Private
-public class PrettyPrinter {
+public final class PrettyPrinter {
 
   private static final Logger LOG = LoggerFactory.getLogger(PrettyPrinter.class);
 
@@ -117,7 +120,7 @@ public class PrettyPrinter {
       sb.append(" DAY").append(days == 1 ? "" : "S");
     }
 
-    if (hours > 0 ) {
+    if (hours > 0) {
       sb.append(days > 0 ? " " : "");
       sb.append(hours);
       sb.append(" HOUR").append(hours == 1 ? "" : "S");
@@ -188,4 +191,18 @@ public class PrettyPrinter {
     return ttl;
   }
 
+  /**
+   * Pretty prints a collection of any type to a string. Relies on toString() implementation of the
+   * object type.
+   * @param collection collection to pretty print.
+   * @return Pretty printed string for the collection.
+   */
+  public static String toString(Collection<?> collection) {
+    List<String> stringList = new ArrayList<>();
+    for (Object o: collection) {
+      stringList.add(Objects.toString(o));
+    }
+    return "[" + String.join(",", stringList) + "]";
+  }
+
 }
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TableNameTestRule.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TableNameTestRule.java
index b565426..626c67e 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TableNameTestRule.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TableNameTestRule.java
@@ -29,7 +29,21 @@ public class TableNameTestRule extends TestWatcher {
 
   @Override
   protected void starting(Description description) {
-    tableName = TableName.valueOf(description.getMethodName());
+    tableName = TableName.valueOf(cleanUpTestName(description.getMethodName()));
+  }
+
+  /**
+   * Helper to handle parameterized method names. Unlike regular test methods, parameterized method
+   * names look like 'foo[x]'. This is problematic for tests that use this name for HBase tables.
+   * This helper strips out the parameter suffixes.
+   * @return current test method name with out parameterized suffixes.
+   */
+  private static String cleanUpTestName(String methodName) {
+    int index = methodName.indexOf('[');
+    if (index == -1) {
+      return methodName;
+    }
+    return methodName.substring(0, index);
   }
 
   public TableName getTableName() {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java
index 5c6ad95..cfa6f75 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java
@@ -26,12 +26,14 @@ import java.util.List;
 import java.util.concurrent.TimeUnit;
 import java.util.function.Supplier;
 
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
 
 /**
  * Utility used running a cluster all in the one JVM.
@@ -136,6 +138,11 @@ public class JVMClusterUtil {
     } catch (Exception e) {
       throw new IOException(e);
     }
+    // Needed if a master based registry is configured for internal cluster connections. Here, we
+    // just add the current master host port since we do not know other master addresses up front
+    // in mini cluster tests.
+    c.set(HConstants.MASTER_ADDRS_KEY,
+        Preconditions.checkNotNull(server.getServerName().getAddress()).toString());
     return new JVMClusterUtil.MasterThread(server, index);
   }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
index 3f40130..d2f7487 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
@@ -1116,6 +1116,9 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
     this.hbaseCluster =
         new MiniHBaseCluster(c, option.getNumMasters(), option.getNumRegionServers(),
             option.getRsPorts(), option.getMasterClass(), option.getRsClass());
+    // Populate the master address configuration from mini cluster configuration.
+    conf.set(HConstants.MASTER_ADDRS_KEY,
+        c.get(HConstants.MASTER_ADDRS_KEY, HConstants.MASTER_ADDRS_DEFAULT));
     // Don't leave here till we've done a successful scan of the hbase:meta
     Table t = getConnection().getTable(TableName.META_TABLE_NAME);
     ResultScanner s = t.getScanner(new Scan());
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/DummyConnectionRegistry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/DummyConnectionRegistry.java
index a669362..c9d67f4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/DummyConnectionRegistry.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/DummyConnectionRegistry.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hbase.client;
 
 import java.util.concurrent.CompletableFuture;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.RegionLocations;
 import org.apache.hadoop.hbase.ServerName;
 
@@ -28,7 +29,7 @@ import org.apache.hadoop.hbase.ServerName;
 public class DummyConnectionRegistry implements ConnectionRegistry {
 
   public static final String REGISTRY_IMPL_CONF_KEY =
-      ConnectionRegistryFactory.CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY;
+      HConstants.CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY;
 
   @Override
   public CompletableFuture<RegionLocations> getMetaRegionLocations() {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
index 61d78ce..f4d81d7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.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,10 +28,10 @@ import static org.junit.Assert.assertSame;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
-
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -64,7 +64,9 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.PrivateCellUtil;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.StartMiniClusterOption;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.TableNameTestRule;
 import org.apache.hadoop.hbase.Waiter;
 import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
@@ -106,23 +108,28 @@ import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.NonRepeatedEnvironmentEdge;
 import org.apache.hadoop.hbase.util.TableDescriptorChecker;
 import org.junit.AfterClass;
-import org.junit.BeforeClass;
+import org.junit.Assume;
 import org.junit.ClassRule;
 import org.junit.Ignore;
 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.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
 
 /**
  * Run tests that use the HBase clients; {@link Table}.
  * Sets up the HBase mini cluster once at start and runs through all client tests.
  * Each creates a table named for the method and does its stuff against that.
+ *
+ * Parameterized to run with different registry implementations.
  */
 @Category({LargeTests.class, ClientTests.class})
 @SuppressWarnings ("deprecation")
+@RunWith(Parameterized.class)
 public class TestFromClientSide {
 
   @ClassRule
@@ -131,7 +138,7 @@ public class TestFromClientSide {
 
   // NOTE: Increment tests were moved to their own class, TestIncrementsFromClientSide.
   private static final Logger LOG = LoggerFactory.getLogger(TestFromClientSide.class);
-  protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  protected static HBaseTestingUtility TEST_UTIL;
   private static byte [] ROW = Bytes.toBytes("testRow");
   private static byte [] FAMILY = Bytes.toBytes("testFamily");
   private static final byte[] INVALID_FAMILY = Bytes.toBytes("invalidTestFamily");
@@ -139,10 +146,54 @@ public class TestFromClientSide {
   private static byte [] VALUE = Bytes.toBytes("testValue");
   protected static int SLAVES = 3;
 
-  @Rule
-  public TestName name = new TestName();
+  @Rule public TableNameTestRule name = new TableNameTestRule();
+
+  // To keep the child classes happy.
+  TestFromClientSide() {}
+
+  public TestFromClientSide(Class registry, int numHedgedReqs) throws Exception {
+    initialize(registry, numHedgedReqs, MultiRowMutationEndpoint.class);
+  }
+
+  @Parameterized.Parameters
+  public static Collection parameters() {
+    return Arrays.asList(new Object[][] {
+        { MasterRegistry.class, 1},
+        { MasterRegistry.class, 2},
+        { ZKConnectionRegistry.class, 1}
+    });
+  }
+
+  /**
+   * JUnit does not provide an easy way to run a hook after each parameterized run. Without that
+   * there is no easy way to restart the test cluster after each parameterized run. Annotation
+   * BeforeParam does not work either because it runs before parameterization and hence does not
+   * have access to the test parameters (which is weird).
+   *
+   * This *hack* checks if the current instance of test cluster configuration has the passed
+   * parameterized configs. In such a case, we can just reuse the cluster for test and do not need
+   * to initialize from scratch. While this is a hack, it saves a ton of time for the full
+   * test and de-flakes it.
+   */
+  private static boolean isSameParameterizedCluster(Class registryImpl, int numHedgedReqs) {
+    if (TEST_UTIL == null) {
+      return false;
+    }
+    Configuration conf = TEST_UTIL.getConfiguration();
+    Class confClass = conf.getClass(
+        HConstants.CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY, ZKConnectionRegistry.class);
+    int hedgedReqConfig = conf.getInt(HConstants.HBASE_RPCS_HEDGED_REQS_FANOUT_KEY,
+        HConstants.HBASE_RPCS_HEDGED_REQS_FANOUT_DEFAULT);
+    return confClass.getName().equals(registryImpl.getName()) && numHedgedReqs == hedgedReqConfig;
+  }
 
-  protected static final void initialize(Class<?>... cps) throws Exception {
+  protected static final void initialize(Class registryImpl, int numHedgedReqs, Class<?>... cps)
+      throws Exception {
+    // initialize() is called for every unit test, however we only want to reset the cluster state
+    // at the end of every parameterized run.
+    if (isSameParameterizedCluster(registryImpl, numHedgedReqs)) {
+      return;
+    }
     // Uncomment the following lines if more verbosity is needed for
     // debugging (see HBASE-12285 for details).
     // ((Log4JLogger)RpcServer.LOG).getLogger().setLevel(Level.ALL);
@@ -150,22 +201,35 @@ public class TestFromClientSide {
     // ((Log4JLogger)ScannerCallable.LOG).getLogger().setLevel(Level.ALL);
     // make sure that we do not get the same ts twice, see HBASE-19731 for more details.
     EnvironmentEdgeManager.injectEdge(new NonRepeatedEnvironmentEdge());
+    if (TEST_UTIL != null) {
+      // We reached end of a parameterized run, clean up.
+      TEST_UTIL.shutdownMiniCluster();
+    }
+    TEST_UTIL = new HBaseTestingUtility();
     Configuration conf = TEST_UTIL.getConfiguration();
     conf.setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,
       Arrays.stream(cps).map(Class::getName).toArray(String[]::new));
     conf.setBoolean(TableDescriptorChecker.TABLE_SANITY_CHECKS, true); // enable for below tests
-    // We need more than one region server in this test
-    TEST_UTIL.startMiniCluster(SLAVES);
-  }
-
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    initialize(MultiRowMutationEndpoint.class);
+    conf.setClass(HConstants.CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY, registryImpl,
+        ConnectionRegistry.class);
+    if (numHedgedReqs == 1) {
+      conf.setBoolean(HConstants.MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY, false);
+    } else {
+      Preconditions.checkArgument(numHedgedReqs > 1);
+      conf.setBoolean(HConstants.MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY, true);
+    }
+    conf.setInt(HConstants.HBASE_RPCS_HEDGED_REQS_FANOUT_KEY, numHedgedReqs);
+    StartMiniClusterOption.Builder builder = StartMiniClusterOption.builder();
+    // Multiple masters needed only when hedged reads for master registry are enabled.
+    builder.numMasters(numHedgedReqs > 1 ? 3 : 1).numRegionServers(SLAVES);
+    TEST_UTIL.startMiniCluster(builder.build());
   }
 
   @AfterClass
   public static void tearDownAfterClass() throws Exception {
-    TEST_UTIL.shutdownMiniCluster();
+    if (TEST_UTIL != null) {
+      TEST_UTIL.shutdownMiniCluster();
+    }
   }
 
   /**
@@ -173,7 +237,7 @@ public class TestFromClientSide {
    */
   @Test
   public void testDuplicateAppend() throws Exception {
-    HTableDescriptor hdt = TEST_UTIL.createTableDescriptor(name.getMethodName());
+    HTableDescriptor hdt = TEST_UTIL.createTableDescriptor(name.getTableName());
     Map<String, String> kvs = new HashMap<>();
     kvs.put(SleepAtFirstRpcCall.SLEEP_TIME_CONF_KEY, "2000");
     hdt.addCoprocessor(SleepAtFirstRpcCall.class.getName(), null, 1, kvs);
@@ -181,11 +245,11 @@ public class TestFromClientSide {
 
     Configuration c = new Configuration(TEST_UTIL.getConfiguration());
     c.setInt(HConstants.HBASE_CLIENT_PAUSE, 50);
-    // Client will retry beacuse rpc timeout is small than the sleep time of first rpc call
+    // Client will retry because rpc timeout is small than the sleep time of first rpc call
     c.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, 1500);
 
     try (Connection connection = ConnectionFactory.createConnection(c);
-        Table table = connection.getTableBuilder(TableName.valueOf(name.getMethodName()), null)
+        Table table = connection.getTableBuilder(name.getTableName(), null)
           .setOperationTimeout(3 * 1000).build()) {
       Append append = new Append(ROW);
       append.addColumn(HBaseTestingUtility.fam1, QUALIFIER, VALUE);
@@ -209,7 +273,7 @@ public class TestFromClientSide {
    */
   @Test
   public void testKeepDeletedCells() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     final byte[] FAMILY = Bytes.toBytes("family");
     final byte[] C0 = Bytes.toBytes("c0");
 
@@ -275,7 +339,7 @@ public class TestFromClientSide {
    */
   @Test
   public void testPurgeFutureDeletes() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     final byte[] ROW = Bytes.toBytes("row");
     final byte[] FAMILY = Bytes.toBytes("family");
     final byte[] COLUMN = Bytes.toBytes("column");
@@ -328,7 +392,7 @@ public class TestFromClientSide {
    */
   @Test
   public void testGetConfiguration() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     byte[][] FAMILIES = new byte[][] { Bytes.toBytes("foo") };
     Configuration conf = TEST_UTIL.getConfiguration();
     try (Table table = TEST_UTIL.createTable(tableName, FAMILIES)) {
@@ -342,7 +406,7 @@ public class TestFromClientSide {
    */
   @Test
   public void testWeirdCacheBehaviour() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     byte [][] FAMILIES = new byte[][] { Bytes.toBytes("trans-blob"),
         Bytes.toBytes("trans-type"), Bytes.toBytes("trans-date"),
         Bytes.toBytes("trans-tags"), Bytes.toBytes("trans-group") };
@@ -383,7 +447,7 @@ public class TestFromClientSide {
   }
 
   private void deleteColumns(Table ht, String value, String keyPrefix)
-  throws IOException {
+      throws IOException {
     ResultScanner scanner = buildScanner(keyPrefix, value, ht);
     Iterator<Result> it = scanner.iterator();
     int count = 0;
@@ -468,8 +532,8 @@ public class TestFromClientSide {
    */
   @Test
   public void testFilterAcrossMultipleRegions()
-  throws IOException, InterruptedException {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+      throws IOException, InterruptedException {
+    final TableName tableName = name.getTableName();
     try (Table t = TEST_UTIL.createTable(tableName, FAMILY)) {
       int rowCount = TEST_UTIL.loadTable(t, FAMILY, false);
       assertRowCount(t, rowCount);
@@ -555,8 +619,7 @@ public class TestFromClientSide {
    * @param t Table to split.
    * @return Map of regions to servers.
    */
-  private List<HRegionLocation> splitTable(final Table t)
-  throws IOException, InterruptedException {
+  private List<HRegionLocation> splitTable(final Table t) throws IOException {
     // Split this table in two.
     Admin admin = TEST_UTIL.getAdmin();
     admin.split(t.getName());
@@ -572,8 +635,7 @@ public class TestFromClientSide {
    * @param t
    * @return Map of table regions; caller needs to check table actually split.
    */
-  private List<HRegionLocation> waitOnSplit(final Table t)
-  throws IOException {
+  private List<HRegionLocation> waitOnSplit(final Table t) throws IOException {
     try (RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(t.getName())) {
       List<HRegionLocation> regions = locator.getAllRegionLocations();
       int originalCount = regions.size();
@@ -585,8 +647,9 @@ public class TestFromClientSide {
           e.printStackTrace();
         }
         regions = locator.getAllRegionLocations();
-        if (regions.size() > originalCount)
+        if (regions.size() > originalCount) {
           break;
+        }
       }
       return regions;
     }
@@ -594,7 +657,7 @@ public class TestFromClientSide {
 
   @Test
   public void testSuperSimple() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     try (Table ht = TEST_UTIL.createTable(tableName, FAMILY)) {
       Put put = new Put(ROW);
       put.addColumn(FAMILY, QUALIFIER, VALUE);
@@ -610,7 +673,7 @@ public class TestFromClientSide {
 
   @Test
   public void testMaxKeyValueSize() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     Configuration conf = TEST_UTIL.getConfiguration();
     String oldMaxSize = conf.get(ConnectionConfiguration.MAX_KEYVALUE_SIZE_KEY);
     try (Table ht = TEST_UTIL.createTable(tableName, FAMILY)) {
@@ -640,7 +703,7 @@ public class TestFromClientSide {
 
   @Test
   public void testFilters() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     try (Table ht = TEST_UTIL.createTable(tableName, FAMILY)) {
       byte[][] ROWS = makeN(ROW, 10);
       byte[][] QUALIFIERS = {
@@ -677,7 +740,7 @@ public class TestFromClientSide {
 
   @Test
   public void testFilterWithLongCompartor() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     try (Table ht = TEST_UTIL.createTable(tableName, FAMILY)) {
       byte[][] ROWS = makeN(ROW, 10);
       byte[][] values = new byte[10][];
@@ -709,7 +772,7 @@ public class TestFromClientSide {
 
   @Test
   public void testKeyOnlyFilter() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     try (Table ht = TEST_UTIL.createTable(tableName, FAMILY)) {
       byte[][] ROWS = makeN(ROW, 10);
       byte[][] QUALIFIERS = {
@@ -747,7 +810,7 @@ public class TestFromClientSide {
    */
   @Test
   public void testSimpleMissing() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     try (Table ht = TEST_UTIL.createTable(tableName, FAMILY)) {
       byte[][] ROWS = makeN(ROW, 4);
 
@@ -858,7 +921,7 @@ public class TestFromClientSide {
    */
   @Test
   public void testSingleRowMultipleFamily() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     byte [][] ROWS = makeN(ROW, 3);
     byte [][] FAMILIES = makeNAscii(FAMILY, 10);
     byte [][] QUALIFIERS = makeN(QUALIFIER, 10);
@@ -1166,7 +1229,7 @@ public class TestFromClientSide {
 
   @Test(expected = IllegalArgumentException.class)
   public void testNullFamilyName() throws IOException {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
 
     // Null family (should NOT work)
     TEST_UTIL.createTable(tableName, new byte[][]{null});
@@ -1175,7 +1238,7 @@ public class TestFromClientSide {
 
   @Test
   public void testNullRowAndQualifier() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
 
     try (Table ht = TEST_UTIL.createTable(tableName, FAMILY)) {
 
@@ -1211,7 +1274,7 @@ public class TestFromClientSide {
 
   @Test
   public void testNullEmptyQualifier() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
 
     try (Table ht = TEST_UTIL.createTable(tableName, FAMILY)) {
 
@@ -1249,7 +1312,7 @@ public class TestFromClientSide {
 
   @Test
   public void testNullValue() throws IOException {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
 
     try (Table ht = TEST_UTIL.createTable(tableName, FAMILY)) {
       // Null value
@@ -1284,7 +1347,7 @@ public class TestFromClientSide {
 
   @Test
   public void testNullQualifier() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     try (Table table = TEST_UTIL.createTable(tableName, FAMILY)) {
 
       // Work for Put
@@ -1342,7 +1405,7 @@ public class TestFromClientSide {
 
   @Test
   public void testVersions() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
 
     long [] STAMPS = makeStamps(20);
     byte [][] VALUES = makeNAscii(VALUE, 20);
@@ -1569,7 +1632,7 @@ public class TestFromClientSide {
   @Test
   @SuppressWarnings("checkstyle:MethodLength")
   public void testVersionLimits() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     byte [][] FAMILIES = makeNAscii(FAMILY, 3);
     int [] LIMITS = {1,3,5};
     long [] STAMPS = makeStamps(10);
@@ -1764,7 +1827,7 @@ public class TestFromClientSide {
   @Test
   public void testDeleteFamilyVersion() throws Exception {
     try (Admin admin = TEST_UTIL.getAdmin()) {
-      final TableName tableName = TableName.valueOf(name.getMethodName());
+      final TableName tableName = name.getTableName();
 
       byte[][] QUALIFIERS = makeNAscii(QUALIFIER, 1);
       byte[][] VALUES = makeN(VALUE, 5);
@@ -1804,7 +1867,7 @@ public class TestFromClientSide {
 
   @Test
   public void testDeleteFamilyVersionWithOtherDeletes() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
 
     byte [][] QUALIFIERS = makeNAscii(QUALIFIER, 5);
     byte [][] VALUES = makeN(VALUE, 5);
@@ -1922,7 +1985,7 @@ public class TestFromClientSide {
 
   @Test
   public void testDeleteWithFailed() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
 
     byte [][] FAMILIES = makeNAscii(FAMILY, 3);
     byte [][] VALUES = makeN(VALUE, 5);
@@ -1948,7 +2011,7 @@ public class TestFromClientSide {
 
   @Test
   public void testDeletes() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
 
     byte [][] ROWS = makeNAscii(ROW, 6);
     byte [][] FAMILIES = makeNAscii(FAMILY, 3);
@@ -2254,7 +2317,7 @@ public class TestFromClientSide {
    */
   @Test
   public void testBatchOperationsWithErrors() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     try (Table foo = TEST_UTIL.createTable(tableName, new byte[][] { FAMILY }, 10)) {
 
       int NUM_OPS = 100;
@@ -2380,7 +2443,7 @@ public class TestFromClientSide {
     int numRows = 10;
     int numColsPerRow = 2000;
 
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
 
     byte [][] ROWS = makeN(ROW, numRows);
     byte [][] QUALIFIERS = makeN(QUALIFIER, numColsPerRow);
@@ -2463,7 +2526,7 @@ public class TestFromClientSide {
    */
   @Test
   public void testJiraTest861() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     byte [][] VALUES = makeNAscii(VALUE, 7);
     long [] STAMPS = makeStamps(7);
 
@@ -2526,7 +2589,7 @@ public class TestFromClientSide {
    */
   @Test
   public void testJiraTest33() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     byte [][] VALUES = makeNAscii(VALUE, 7);
     long [] STAMPS = makeStamps(7);
 
@@ -2574,7 +2637,7 @@ public class TestFromClientSide {
    */
   @Test
   public void testJiraTest1014() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
 
     try (Table ht = TEST_UTIL.createTable(tableName, FAMILY, 10)) {
 
@@ -2598,7 +2661,7 @@ public class TestFromClientSide {
    */
   @Test
   public void testJiraTest1182() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     byte [][] VALUES = makeNAscii(VALUE, 7);
     long [] STAMPS = makeStamps(7);
 
@@ -2642,7 +2705,7 @@ public class TestFromClientSide {
    */
   @Test
   public void testJiraTest52() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     byte [][] VALUES = makeNAscii(VALUE, 7);
     long [] STAMPS = makeStamps(7);
 
@@ -2678,8 +2741,7 @@ public class TestFromClientSide {
 
   private void getVersionRangeAndVerifyGreaterThan(Table ht, byte [] row,
       byte [] family, byte [] qualifier, long [] stamps, byte [][] values,
-      int start, int end)
-  throws IOException {
+      int start, int end) throws IOException {
     Get get = new Get(row);
     get.addColumn(family, qualifier);
     get.readVersions(Integer.MAX_VALUE);
@@ -2689,8 +2751,7 @@ public class TestFromClientSide {
   }
 
   private void getVersionRangeAndVerify(Table ht, byte [] row, byte [] family,
-      byte [] qualifier, long [] stamps, byte [][] values, int start, int end)
-  throws IOException {
+      byte [] qualifier, long [] stamps, byte [][] values, int start, int end) throws IOException {
     Get get = new Get(row);
     get.addColumn(family, qualifier);
     get.readVersions(Integer.MAX_VALUE);
@@ -2700,8 +2761,7 @@ public class TestFromClientSide {
   }
 
   private void getAllVersionsAndVerify(Table ht, byte [] row, byte [] family,
-      byte [] qualifier, long [] stamps, byte [][] values, int start, int end)
-  throws IOException {
+      byte [] qualifier, long [] stamps, byte [][] values, int start, int end) throws IOException {
     Get get = new Get(row);
     get.addColumn(family, qualifier);
     get.readVersions(Integer.MAX_VALUE);
@@ -2711,8 +2771,7 @@ public class TestFromClientSide {
 
   private void scanVersionRangeAndVerifyGreaterThan(Table ht, byte [] row,
       byte [] family, byte [] qualifier, long [] stamps, byte [][] values,
-      int start, int end)
-  throws IOException {
+      int start, int end) throws IOException {
     Scan scan = new Scan(row);
     scan.addColumn(family, qualifier);
     scan.setMaxVersions(Integer.MAX_VALUE);
@@ -2722,8 +2781,7 @@ public class TestFromClientSide {
   }
 
   private void scanVersionRangeAndVerify(Table ht, byte [] row, byte [] family,
-      byte [] qualifier, long [] stamps, byte [][] values, int start, int end)
-  throws IOException {
+      byte [] qualifier, long [] stamps, byte [][] values, int start, int end) throws IOException {
     Scan scan = new Scan(row);
     scan.addColumn(family, qualifier);
     scan.setMaxVersions(Integer.MAX_VALUE);
@@ -2733,8 +2791,7 @@ public class TestFromClientSide {
   }
 
   private void scanAllVersionsAndVerify(Table ht, byte [] row, byte [] family,
-      byte [] qualifier, long [] stamps, byte [][] values, int start, int end)
-  throws IOException {
+      byte [] qualifier, long [] stamps, byte [][] values, int start, int end) throws IOException {
     Scan scan = new Scan(row);
     scan.addColumn(family, qualifier);
     scan.setMaxVersions(Integer.MAX_VALUE);
@@ -2743,8 +2800,7 @@ public class TestFromClientSide {
   }
 
   private void getVersionAndVerify(Table ht, byte [] row, byte [] family,
-      byte [] qualifier, long stamp, byte [] value)
-  throws Exception {
+      byte [] qualifier, long stamp, byte [] value) throws Exception {
     Get get = new Get(row);
     get.addColumn(family, qualifier);
     get.setTimestamp(stamp);
@@ -2754,8 +2810,7 @@ public class TestFromClientSide {
   }
 
   private void getVersionAndVerifyMissing(Table ht, byte [] row, byte [] family,
-      byte [] qualifier, long stamp)
-  throws Exception {
+      byte [] qualifier, long stamp) throws Exception {
     Get get = new Get(row);
     get.addColumn(family, qualifier);
     get.setTimestamp(stamp);
@@ -2765,8 +2820,7 @@ public class TestFromClientSide {
   }
 
   private void scanVersionAndVerify(Table ht, byte [] row, byte [] family,
-      byte [] qualifier, long stamp, byte [] value)
-  throws Exception {
+      byte [] qualifier, long stamp, byte [] value) throws Exception {
     Scan scan = new Scan(row);
     scan.addColumn(family, qualifier);
     scan.setTimestamp(stamp);
@@ -2776,8 +2830,7 @@ public class TestFromClientSide {
   }
 
   private void scanVersionAndVerifyMissing(Table ht, byte [] row,
-      byte [] family, byte [] qualifier, long stamp)
-  throws Exception {
+      byte [] family, byte [] qualifier, long stamp) throws Exception {
     Scan scan = new Scan(row);
     scan.addColumn(family, qualifier);
     scan.setTimestamp(stamp);
@@ -2786,10 +2839,7 @@ public class TestFromClientSide {
     assertNullResult(result);
   }
 
-  private void getTestNull(Table ht, byte [] row, byte [] family,
-      byte [] value)
-  throws Exception {
-
+  private void getTestNull(Table ht, byte [] row, byte [] family, byte [] value) throws Exception {
     Get get = new Get(row);
     get.addColumn(family, null);
     Result result = ht.get(get);
@@ -2866,9 +2916,7 @@ public class TestFromClientSide {
   }
 
   private void singleRowGetTest(Table ht, byte [][] ROWS, byte [][] FAMILIES,
-      byte [][] QUALIFIERS, byte [][] VALUES)
-  throws Exception {
-
+      byte [][] QUALIFIERS, byte [][] VALUES) throws Exception {
     // Single column from memstore
     Get get = new Get(ROWS[0]);
     get.addColumn(FAMILIES[4], QUALIFIERS[0]);
@@ -2923,7 +2971,7 @@ public class TestFromClientSide {
     assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES,
         new int [][] {
           {2, 2, 2}, {2, 4, 4}, {4, 0, 0}, {4, 4, 4}, {6, 6, 6}, {6, 7, 7}, {7, 7, 7}
-    });
+        });
 
     // Multiple columns from everywhere storefile, many family, wildcard
     get = new Get(ROWS[0]);
@@ -2939,7 +2987,7 @@ public class TestFromClientSide {
     assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES,
         new int [][] {
           {2, 2, 2}, {2, 4, 4}, {4, 0, 0}, {4, 4, 4}, {6, 6, 6}, {6, 7, 7}, {7, 7, 7}
-    });
+        });
 
     // Everything
     get = new Get(ROWS[0]);
@@ -2947,7 +2995,7 @@ public class TestFromClientSide {
     assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES,
         new int [][] {
           {2, 2, 2}, {2, 4, 4}, {4, 0, 0}, {4, 4, 4}, {6, 6, 6}, {6, 7, 7}, {7, 7, 7}, {9, 0, 0}
-    });
+        });
 
     // Get around inserted columns
 
@@ -2964,9 +3012,7 @@ public class TestFromClientSide {
   }
 
   private void singleRowScanTest(Table ht, byte [][] ROWS, byte [][] FAMILIES,
-      byte [][] QUALIFIERS, byte [][] VALUES)
-  throws Exception {
-
+      byte [][] QUALIFIERS, byte [][] VALUES) throws Exception {
     // Single column from memstore
     Scan scan = new Scan();
     scan.addColumn(FAMILIES[4], QUALIFIERS[0]);
@@ -3021,7 +3067,7 @@ public class TestFromClientSide {
     assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES,
         new int [][] {
           {2, 2, 2}, {2, 4, 4}, {4, 0, 0}, {4, 4, 4}, {6, 6, 6}, {6, 7, 7}, {7, 7, 7}
-    });
+        });
 
     // Multiple columns from everywhere storefile, many family, wildcard
     scan = new Scan();
@@ -3037,7 +3083,7 @@ public class TestFromClientSide {
     assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES,
         new int [][] {
           {2, 2, 2}, {2, 4, 4}, {4, 0, 0}, {4, 4, 4}, {6, 6, 6}, {6, 7, 7}, {7, 7, 7}
-    });
+        });
 
     // Everything
     scan = new Scan();
@@ -3045,7 +3091,7 @@ public class TestFromClientSide {
     assertNResult(result, ROWS[0], FAMILIES, QUALIFIERS, VALUES,
         new int [][] {
           {2, 2, 2}, {2, 4, 4}, {4, 0, 0}, {4, 4, 4}, {6, 6, 6}, {6, 7, 7}, {7, 7, 7}, {9, 0, 0}
-    });
+        });
 
     // Scan around inserted columns
 
@@ -3065,13 +3111,9 @@ public class TestFromClientSide {
    * Expects family and qualifier arrays to be valid for at least
    * the range:  idx-2 < idx < idx+2
    */
-  private void getVerifySingleColumn(Table ht,
-      byte [][] ROWS, int ROWIDX,
-      byte [][] FAMILIES, int FAMILYIDX,
-      byte [][] QUALIFIERS, int QUALIFIERIDX,
-      byte [][] VALUES, int VALUEIDX)
-  throws Exception {
-
+  private void getVerifySingleColumn(Table ht, byte [][] ROWS, int ROWIDX, byte [][] FAMILIES,
+      int FAMILYIDX, byte [][] QUALIFIERS, int QUALIFIERIDX, byte [][] VALUES, int VALUEIDX)
+      throws Exception {
     Get get = new Get(ROWS[ROWIDX]);
     Result result = ht.get(get);
     assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
@@ -3123,13 +3165,9 @@ public class TestFromClientSide {
    * the range:  idx-2 to idx+2
    * Expects row array to be valid for at least idx to idx+2
    */
-  private void scanVerifySingleColumn(Table ht,
-      byte [][] ROWS, int ROWIDX,
-      byte [][] FAMILIES, int FAMILYIDX,
-      byte [][] QUALIFIERS, int QUALIFIERIDX,
-      byte [][] VALUES, int VALUEIDX)
-  throws Exception {
-
+  private void scanVerifySingleColumn(Table ht, byte [][] ROWS, int ROWIDX, byte [][] FAMILIES,
+      int FAMILYIDX, byte [][] QUALIFIERS, int QUALIFIERIDX, byte [][] VALUES, int VALUEIDX)
+      throws Exception {
     Scan scan = new Scan();
     Result result = getSingleScanResult(ht, scan);
     assertSingleResult(result, ROWS[ROWIDX], FAMILIES[FAMILYIDX],
@@ -3183,12 +3221,8 @@ public class TestFromClientSide {
    * Verify we do not read any values by accident around a single column
    * Same requirements as getVerifySingleColumn
    */
-  private void getVerifySingleEmpty(Table ht,
-      byte [][] ROWS, int ROWIDX,
-      byte [][] FAMILIES, int FAMILYIDX,
-      byte [][] QUALIFIERS, int QUALIFIERIDX)
-  throws Exception {
-
+  private void getVerifySingleEmpty(Table ht, byte [][] ROWS, int ROWIDX, byte [][] FAMILIES,
+      int FAMILYIDX, byte [][] QUALIFIERS, int QUALIFIERIDX) throws Exception {
     Get get = new Get(ROWS[ROWIDX]);
     get.addFamily(FAMILIES[4]);
     get.addColumn(FAMILIES[4], QUALIFIERS[1]);
@@ -3214,12 +3248,8 @@ public class TestFromClientSide {
 
   }
 
-  private void scanVerifySingleEmpty(Table ht,
-      byte [][] ROWS, int ROWIDX,
-      byte [][] FAMILIES, int FAMILYIDX,
-      byte [][] QUALIFIERS, int QUALIFIERIDX)
-  throws Exception {
-
+  private void scanVerifySingleEmpty(Table ht, byte [][] ROWS, int ROWIDX, byte [][] FAMILIES,
+      int FAMILYIDX, byte [][] QUALIFIERS, int QUALIFIERIDX) throws Exception {
     Scan scan = new Scan(ROWS[ROWIDX+1]);
     Result result = getSingleScanResult(ht, scan);
     assertNullResult(result);
@@ -3244,9 +3274,7 @@ public class TestFromClientSide {
   // Verifiers
   //
 
-  private void assertKey(Cell key, byte [] row, byte [] family,
-      byte [] qualifier, byte [] value)
-  throws Exception {
+  private void assertKey(Cell key, byte [] row, byte [] family, byte [] qualifier, byte [] value) {
     assertTrue("Expected row [" + Bytes.toString(row) + "] " +
         "Got row [" + Bytes.toString(CellUtil.cloneRow(key)) +"]",
         equals(row, CellUtil.cloneRow(key)));
@@ -3262,8 +3290,7 @@ public class TestFromClientSide {
   }
 
   static void assertIncrementKey(Cell key, byte [] row, byte [] family,
-      byte [] qualifier, long value)
-  throws Exception {
+      byte [] qualifier, long value) {
     assertTrue("Expected row [" + Bytes.toString(row) + "] " +
         "Got row [" + Bytes.toString(CellUtil.cloneRow(key)) +"]",
         equals(row, CellUtil.cloneRow(key)));
@@ -3284,9 +3311,7 @@ public class TestFromClientSide {
   }
 
   private void assertNResult(Result result, byte [] row,
-      byte [][] families, byte [][] qualifiers, byte [][] values,
-      int [][] idxs)
-  throws Exception {
+      byte [][] families, byte [][] qualifiers, byte [][] values, int [][] idxs) {
     assertTrue("Expected row [" + Bytes.toString(row) + "] " +
         "Got row [" + Bytes.toString(result.getRow()) +"]",
         equals(row, result.getRow()));
@@ -3318,8 +3343,7 @@ public class TestFromClientSide {
 
   private void assertNResult(Result result, byte [] row,
       byte [] family, byte [] qualifier, long [] stamps, byte [][] values,
-      int start, int end)
-  throws IOException {
+      int start, int end) {
     assertTrue("Expected row [" + Bytes.toString(row) + "] " +
         "Got row [" + Bytes.toString(result.getRow()) +"]",
         equals(row, result.getRow()));
@@ -3353,8 +3377,7 @@ public class TestFromClientSide {
    */
   private void assertDoubleResult(Result result, byte [] row,
       byte [] familyA, byte [] qualifierA, byte [] valueA,
-      byte [] familyB, byte [] qualifierB, byte [] valueB)
-  throws Exception {
+      byte [] familyB, byte [] qualifierB, byte [] valueB) {
     assertTrue("Expected row [" + Bytes.toString(row) + "] " +
         "Got row [" + Bytes.toString(result.getRow()) +"]",
         equals(row, result.getRow()));
@@ -3384,8 +3407,7 @@ public class TestFromClientSide {
   }
 
   private void assertSingleResult(Result result, byte [] row, byte [] family,
-      byte [] qualifier, byte [] value)
-  throws Exception {
+      byte [] qualifier, byte [] value) {
     assertTrue("Expected row [" + Bytes.toString(row) + "] " +
         "Got row [" + Bytes.toString(result.getRow()) +"]",
         equals(row, result.getRow()));
@@ -3423,8 +3445,7 @@ public class TestFromClientSide {
   }
 
   private void assertSingleResult(Result result, byte [] row, byte [] family,
-      byte [] qualifier, long ts, byte [] value)
-  throws Exception {
+      byte [] qualifier, long ts, byte [] value) {
     assertTrue("Expected row [" + Bytes.toString(row) + "] " +
         "Got row [" + Bytes.toString(result.getRow()) +"]",
         equals(row, result.getRow()));
@@ -3515,7 +3536,7 @@ public class TestFromClientSide {
 
   @Test
   public void testDuplicateVersions() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
 
     long [] STAMPS = makeStamps(20);
     byte [][] VALUES = makeNAscii(VALUE, 20);
@@ -3738,7 +3759,7 @@ public class TestFromClientSide {
 
   @Test
   public void testUpdates() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     try (Table hTable = TEST_UTIL.createTable(tableName, FAMILY, 10)) {
 
       // Write a column with values at timestamp 1, 2 and 3
@@ -3788,7 +3809,7 @@ public class TestFromClientSide {
 
   @Test
   public void testUpdatesWithMajorCompaction() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     try (Table hTable = TEST_UTIL.createTable(tableName, FAMILY, 10);
         Admin admin = TEST_UTIL.getAdmin()) {
 
@@ -3849,7 +3870,7 @@ public class TestFromClientSide {
 
   @Test
   public void testMajorCompactionBetweenTwoUpdates() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     try (Table hTable = TEST_UTIL.createTable(tableName, FAMILY, 10);
         Admin admin = TEST_UTIL.getAdmin()) {
 
@@ -3916,7 +3937,7 @@ public class TestFromClientSide {
 
   @Test
   public void testGet_EmptyTable() throws IOException {
-    try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()), FAMILY)) {
+    try (Table table = TEST_UTIL.createTable(name.getTableName(), FAMILY)) {
       Get get = new Get(ROW);
       get.addFamily(FAMILY);
       Result r = table.get(get);
@@ -3926,7 +3947,7 @@ public class TestFromClientSide {
 
   @Test
   public void testGet_NullQualifier() throws IOException {
-    try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()), FAMILY)) {
+    try (Table table = TEST_UTIL.createTable(name.getTableName(), FAMILY)) {
       Put put = new Put(ROW);
       put.addColumn(FAMILY, QUALIFIER, VALUE);
       table.put(put);
@@ -3950,7 +3971,7 @@ public class TestFromClientSide {
 
   @Test
   public void testGet_NonExistentRow() throws IOException {
-    try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()), FAMILY)) {
+    try (Table table = TEST_UTIL.createTable(name.getTableName(), FAMILY)) {
       Put put = new Put(ROW);
       put.addColumn(FAMILY, QUALIFIER, VALUE);
       table.put(put);
@@ -3978,7 +3999,7 @@ public class TestFromClientSide {
     final byte [] row1 = Bytes.toBytes("row1");
     final byte [] row2 = Bytes.toBytes("row2");
     final byte [] value = Bytes.toBytes("abcd");
-    try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()),
+    try (Table table = TEST_UTIL.createTable(name.getTableName(),
         new byte[][] { CONTENTS_FAMILY, SMALL_FAMILY })) {
       Put put = new Put(row1);
       put.addColumn(CONTENTS_FAMILY, null, value);
@@ -4017,7 +4038,7 @@ public class TestFromClientSide {
   public void testPutNoCF() throws IOException {
     final byte[] BAD_FAM = Bytes.toBytes("BAD_CF");
     final byte[] VAL = Bytes.toBytes(100);
-    try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()), FAMILY)) {
+    try (Table table = TEST_UTIL.createTable(name.getTableName(), FAMILY)) {
       boolean caughtNSCFE = false;
 
       try {
@@ -4037,7 +4058,7 @@ public class TestFromClientSide {
     final byte[] SMALL_FAMILY = Bytes.toBytes("smallfam");
     final int NB_BATCH_ROWS = 10;
     final byte[] value = Bytes.toBytes("abcd");
-    try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()),
+    try (Table table = TEST_UTIL.createTable(name.getTableName(),
       new byte[][] {CONTENTS_FAMILY, SMALL_FAMILY })) {
       ArrayList<Put> rowsUpdate = new ArrayList<Put>();
       for (int i = 0; i < NB_BATCH_ROWS; i++) {
@@ -4067,7 +4088,7 @@ public class TestFromClientSide {
     final byte[] SMALL_FAMILY = Bytes.toBytes("smallfam");
     final byte[] value = Bytes.toBytes("abcd");
     final int NB_BATCH_ROWS = 10;
-    try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()),
+    try (Table table = TEST_UTIL.createTable(name.getTableName(),
         new byte[][] { CONTENTS_FAMILY, SMALL_FAMILY })) {
       ArrayList<Put> rowsUpdate = new ArrayList<Put>();
       for (int i = 0; i < NB_BATCH_ROWS * 10; i++) {
@@ -4130,7 +4151,7 @@ public class TestFromClientSide {
     final byte [] FAM1 = Bytes.toBytes("fam1");
     final byte [] FAM2 = Bytes.toBytes("fam2");
     // Open table
-    try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()),
+    try (Table table = TEST_UTIL.createTable(name.getTableName(),
       new byte [][] {FAM1, FAM2})) {
       // Insert some values
       Put put = new Put(ROW);
@@ -4213,9 +4234,10 @@ public class TestFromClientSide {
 
   @Test
   public void testListTables() throws IOException, InterruptedException {
-    final TableName tableName1 = TableName.valueOf(name.getMethodName() + "1");
-    final TableName tableName2 = TableName.valueOf(name.getMethodName() + "2");
-    final TableName tableName3 = TableName.valueOf(name.getMethodName() + "3");
+    final String testTableName = name.getTableName().toString();
+    final TableName tableName1 = TableName.valueOf(testTableName + "1");
+    final TableName tableName2 = TableName.valueOf(testTableName + "2");
+    final TableName tableName3 = TableName.valueOf(testTableName + "3");
     TableName [] tables = new TableName[] { tableName1, tableName2, tableName3 };
     for (int i = 0; i < tables.length; i++) {
       TEST_UTIL.createTable(tables[i], FAMILY);
@@ -4244,7 +4266,7 @@ public class TestFromClientSide {
    */
   @Test
   public void testUnmanagedHConnection() throws IOException {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY);
     try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration());
         Table t = conn.getTable(tableName);
@@ -4260,7 +4282,13 @@ public class TestFromClientSide {
    */
   @Test
   public void testUnmanagedHConnectionReconnect() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    Configuration conf = TEST_UTIL.getConfiguration();
+    Class registryImpl = conf.getClass(
+        HConstants.CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY, ZKConnectionRegistry.class);
+    // This test does not make sense for MasterRegistry since it stops the only master in the
+    // cluster and starts a new master without populating the underlying config for the connection.
+    Assume.assumeFalse(registryImpl.equals(MasterRegistry.class));
+    final TableName tableName = name.getTableName();
     TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY);
     try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration())) {
       try (Table t = conn.getTable(tableName); Admin admin = conn.getAdmin()) {
@@ -4290,8 +4318,9 @@ public class TestFromClientSide {
 
   @Test
   public void testMiscHTableStuff() throws IOException {
-    final TableName tableAname = TableName.valueOf(name.getMethodName() + "A");
-    final TableName tableBname = TableName.valueOf(name.getMethodName() + "B");
+    final String testTableName = name.getTableName().toString();
+    final TableName tableAname = TableName.valueOf(testTableName + "A");
+    final TableName tableBname = TableName.valueOf(testTableName + "B");
     final byte[] attrName = Bytes.toBytes("TESTATTR");
     final byte[] attrValue = Bytes.toBytes("somevalue");
     byte[] value = Bytes.toBytes("value");
@@ -4340,8 +4369,9 @@ public class TestFromClientSide {
       // add a user attribute to HTD
       desc.setValue(attrName, attrValue);
       // add a user attribute to HCD
-      for (HColumnDescriptor c : desc.getFamilies())
+      for (HColumnDescriptor c : desc.getFamilies()) {
         c.setValue(attrName, attrValue);
+      }
       // update metadata for all regions of this table
       admin.modifyTable(desc);
       // enable the table
@@ -4365,7 +4395,7 @@ public class TestFromClientSide {
 
   @Test
   public void testGetClosestRowBefore() throws IOException, InterruptedException {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     final byte[] firstRow = Bytes.toBytes("row111");
     final byte[] secondRow = Bytes.toBytes("row222");
     final byte[] thirdRow = Bytes.toBytes("row333");
@@ -4492,7 +4522,7 @@ public class TestFromClientSide {
   @Test
   public void testMultiRowMutation() throws Exception {
     LOG.info("Starting testMultiRowMutation");
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     final byte [] ROW1 = Bytes.toBytes("testRow1");
 
     try (Table t = TEST_UTIL.createTable(tableName, FAMILY)) {
@@ -4524,7 +4554,7 @@ public class TestFromClientSide {
   @Test
   public void testRowMutation() throws Exception {
     LOG.info("Starting testRowMutation");
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     try (Table t = TEST_UTIL.createTable(tableName, FAMILY)) {
       byte[][] QUALIFIERS = new byte[][] { Bytes.toBytes("a"), Bytes.toBytes("b") };
       RowMutations arm = new RowMutations(ROW);
@@ -4574,7 +4604,7 @@ public class TestFromClientSide {
   @Test
   public void testBatchAppendWithReturnResultFalse() throws Exception {
     LOG.info("Starting testBatchAppendWithReturnResultFalse");
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     try (Table table = TEST_UTIL.createTable(tableName, FAMILY)) {
       Append append1 = new Append(Bytes.toBytes("row1"));
       append1.setReturnResults(false);
@@ -4598,7 +4628,7 @@ public class TestFromClientSide {
   @Test
   public void testAppend() throws Exception {
     LOG.info("Starting testAppend");
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     try (Table t = TEST_UTIL.createTable(tableName, FAMILY)) {
       byte[] v1 = Bytes.toBytes("42");
       byte[] v2 = Bytes.toBytes("23");
@@ -4690,7 +4720,7 @@ public class TestFromClientSide {
 
   @Test
   public void testClientPoolRoundRobin() throws IOException {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
 
     int poolSize = 3;
     int numVersions = poolSize * 2;
@@ -4728,7 +4758,7 @@ public class TestFromClientSide {
 
   @Ignore ("Flakey: HBASE-8989") @Test
   public void testClientPoolThreadLocal() throws IOException {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
 
     int poolSize = Integer.MAX_VALUE;
     int numVersions = 3;
@@ -4814,7 +4844,7 @@ public class TestFromClientSide {
     final byte [] anotherrow = Bytes.toBytes("anotherrow");
     final byte [] value2 = Bytes.toBytes("abcd");
 
-    try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()), FAMILY)) {
+    try (Table table = TEST_UTIL.createTable(name.getTableName(), FAMILY)) {
       Put put1 = new Put(ROW);
       put1.addColumn(FAMILY, QUALIFIER, VALUE);
 
@@ -4852,7 +4882,7 @@ public class TestFromClientSide {
 
   @Test
   public void testCheckAndMutateWithTimeRange() throws IOException {
-    try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()), FAMILY)) {
+    try (Table table = TEST_UTIL.createTable(name.getTableName(), FAMILY)) {
       final long ts = System.currentTimeMillis() / 2;
       Put put = new Put(ROW);
       put.addColumn(FAMILY, QUALIFIER, ts, VALUE);
@@ -4948,7 +4978,7 @@ public class TestFromClientSide {
     final byte [] value3 = Bytes.toBytes("cccc");
     final byte [] value4 = Bytes.toBytes("dddd");
 
-    try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()), FAMILY)) {
+    try (Table table = TEST_UTIL.createTable(name.getTableName(), FAMILY)) {
 
       Put put2 = new Put(ROW);
       put2.addColumn(FAMILY, QUALIFIER, value2);
@@ -5030,7 +5060,7 @@ public class TestFromClientSide {
   public void testCheckAndDelete() throws IOException {
     final byte [] value1 = Bytes.toBytes("aaaa");
 
-    try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()),
+    try (Table table = TEST_UTIL.createTable(name.getTableName(),
         FAMILY)) {
 
       Put put = new Put(ROW);
@@ -5053,7 +5083,7 @@ public class TestFromClientSide {
     final byte [] value3 = Bytes.toBytes("cccc");
     final byte [] value4 = Bytes.toBytes("dddd");
 
-    try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()),
+    try (Table table = TEST_UTIL.createTable(name.getTableName(),
         FAMILY)) {
 
       Put put2 = new Put(ROW);
@@ -5143,9 +5173,9 @@ public class TestFromClientSide {
   * Test ScanMetrics
   */
   @Test
-  @SuppressWarnings ("unused")
+  @SuppressWarnings({"unused", "checkstyle:EmptyBlock"})
   public void testScanMetrics() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
 
     // Set up test table:
     // Create table:
@@ -5198,7 +5228,6 @@ public class TestFromClientSide {
         // the end of the scanner. So this is asking for 2 of the 3 rows we inserted.
         for (Result result : scanner.next(numRecords - 1)) {
         }
-
         ScanMetrics scanMetrics = scanner.getScanMetrics();
         assertEquals("Did not access all the regions in the table", numOfRegions,
                 scanMetrics.countOfRegions.get());
@@ -5279,7 +5308,7 @@ public class TestFromClientSide {
    */
   @Test
   public void testCacheOnWriteEvictOnClose() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     byte [] data = Bytes.toBytes("data");
     try (Table table = TEST_UTIL.createTable(tableName, FAMILY)) {
       try (RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
@@ -5403,7 +5432,7 @@ public class TestFromClientSide {
    */
   public void testNonCachedGetRegionLocation() throws Exception {
     // Test Initialization.
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     byte [] family1 = Bytes.toBytes("f1");
     byte [] family2 = Bytes.toBytes("f2");
     try (Table table = TEST_UTIL.createTable(tableName, new byte[][] {family1, family2}, 10);
@@ -5452,7 +5481,7 @@ public class TestFromClientSide {
     // Test Initialization.
     byte [] startKey = Bytes.toBytes("ddc");
     byte [] endKey = Bytes.toBytes("mmm");
-    TableName tableName = TableName.valueOf(name.getMethodName());
+    TableName tableName = name.getTableName();
     TEST_UTIL.createMultiRegionTable(tableName, new byte[][] { FAMILY }, 10);
 
     int numOfRegions = -1;
@@ -5522,7 +5551,7 @@ public class TestFromClientSide {
 
   @Test
   public void testJira6912() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     try (Table foo = TEST_UTIL.createTable(tableName, new byte[][] {FAMILY}, 10)) {
 
       List<Put> puts = new ArrayList<Put>();
@@ -5551,7 +5580,7 @@ public class TestFromClientSide {
 
   @Test
   public void testScan_NullQualifier() throws IOException {
-    try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()), FAMILY)) {
+    try (Table table = TEST_UTIL.createTable(name.getTableName(), FAMILY)) {
       Put put = new Put(ROW);
       put.addColumn(FAMILY, QUALIFIER, VALUE);
       table.put(put);
@@ -5581,7 +5610,7 @@ public class TestFromClientSide {
 
   @Test
   public void testNegativeTimestamp() throws IOException {
-    try (Table table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()), FAMILY)) {
+    try (Table table = TEST_UTIL.createTable(name.getTableName(), FAMILY)) {
 
       try {
         Put put = new Put(ROW, -1);
@@ -5642,7 +5671,7 @@ public class TestFromClientSide {
 
   @Test
   public void testRawScanRespectsVersions() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     try (Table table = TEST_UTIL.createTable(tableName, FAMILY)) {
       byte[] row = Bytes.toBytes("row");
 
@@ -5716,7 +5745,7 @@ public class TestFromClientSide {
   @Test
   public void testEmptyFilterList() throws Exception {
     // Test Initialization.
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     try (Table table = TEST_UTIL.createTable(tableName, FAMILY)) {
 
       // Insert one row each region
@@ -5756,7 +5785,7 @@ public class TestFromClientSide {
   @Test
   public void testSmallScan() throws Exception {
     // Test Initialization.
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     try (Table table = TEST_UTIL.createTable(tableName, FAMILY)) {
 
       // Insert one row each region
@@ -5794,7 +5823,7 @@ public class TestFromClientSide {
 
   @Test
   public void testSuperSimpleWithReverseScan() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     try (Table ht = TEST_UTIL.createTable(tableName, FAMILY)) {
       Put put = new Put(Bytes.toBytes("0-b11111-0000000000000000000"));
       put.addColumn(FAMILY, QUALIFIER, VALUE);
@@ -5839,7 +5868,7 @@ public class TestFromClientSide {
 
   @Test
   public void testFiltersWithReverseScan() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     try (Table ht = TEST_UTIL.createTable(tableName, FAMILY)) {
       byte[][] ROWS = makeN(ROW, 10);
       byte[][] QUALIFIERS = {Bytes.toBytes("col0-<d2v1>-<d3v2>"),
@@ -5882,7 +5911,7 @@ public class TestFromClientSide {
 
   @Test
   public void testKeyOnlyFilterWithReverseScan() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     try (Table ht = TEST_UTIL.createTable(tableName, FAMILY)) {
       byte[][] ROWS = makeN(ROW, 10);
       byte[][] QUALIFIERS = {Bytes.toBytes("col0-<d2v1>-<d3v2>"),
@@ -5923,7 +5952,7 @@ public class TestFromClientSide {
    */
   @Test
   public void testSimpleMissingWithReverseScan() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     try (Table ht = TEST_UTIL.createTable(tableName, FAMILY)) {
       byte[][] ROWS = makeN(ROW, 4);
 
@@ -5988,7 +6017,7 @@ public class TestFromClientSide {
 
   @Test
   public void testNullWithReverseScan() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     try (Table ht = TEST_UTIL.createTable(tableName, FAMILY)) {
       // Null qualifier (should work)
       Put put = new Put(ROW);
@@ -6001,7 +6030,8 @@ public class TestFromClientSide {
     }
 
     // Use a new table
-    try (Table ht = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName() + "2"), FAMILY)) {
+    try (Table ht =
+       TEST_UTIL.createTable(TableName.valueOf(name.getTableName().toString() + "2"), FAMILY)) {
       // Empty qualifier, byte[0] instead of null (should work)
       Put put = new Put(ROW);
       put.addColumn(FAMILY, HConstants.EMPTY_BYTE_ARRAY, VALUE);
@@ -6027,7 +6057,7 @@ public class TestFromClientSide {
   @Test
   @SuppressWarnings("checkstyle:MethodLength")
   public void testDeletesWithReverseScan() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     byte[][] ROWS = makeNAscii(ROW, 6);
     byte[][] FAMILIES = makeNAscii(FAMILY, 3);
     byte[][] VALUES = makeN(VALUE, 5);
@@ -6213,7 +6243,7 @@ public class TestFromClientSide {
   @Test
   public void testReversedScanUnderMultiRegions() throws Exception {
     // Test Initialization.
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     byte[] maxByteArray = ConnectionUtils.MAX_BYTE_ARRAY;
     byte[][] splitRows = new byte[][] { Bytes.toBytes("005"),
         Bytes.add(Bytes.toBytes("005"), Bytes.multiple(maxByteArray, 16)),
@@ -6274,7 +6304,7 @@ public class TestFromClientSide {
   @Test
   public void testSmallReversedScanUnderMultiRegions() throws Exception {
     // Test Initialization.
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     byte[][] splitRows = new byte[][]{
         Bytes.toBytes("000"), Bytes.toBytes("002"), Bytes.toBytes("004"),
         Bytes.toBytes("006"), Bytes.toBytes("008"), Bytes.toBytes("010")};
@@ -6494,7 +6524,7 @@ public class TestFromClientSide {
   @Test
   public void testDeleteSpecifiedVersionOfSpecifiedColumn() throws Exception {
     try (Admin admin = TEST_UTIL.getAdmin()) {
-      final TableName tableName = TableName.valueOf(name.getMethodName());
+      final TableName tableName = name.getTableName();
 
       byte[][] VALUES = makeN(VALUE, 5);
       long[] ts = {1000, 2000, 3000, 4000, 5000};
@@ -6540,7 +6570,7 @@ public class TestFromClientSide {
   @Test
   public void testDeleteLatestVersionOfSpecifiedColumn() throws Exception {
     try (Admin admin = TEST_UTIL.getAdmin()) {
-      final TableName tableName = TableName.valueOf(name.getMethodName());
+      final TableName tableName = name.getTableName();
 
       byte[][] VALUES = makeN(VALUE, 5);
       long[] ts = {1000, 2000, 3000, 4000, 5000};
@@ -6603,7 +6633,7 @@ public class TestFromClientSide {
   @Test
   public void testReadWithFilter() throws Exception {
     try (Admin admin = TEST_UTIL.getAdmin()) {
-      final TableName tableName = TableName.valueOf(name.getMethodName());
+      final TableName tableName = name.getTableName();
       try (Table table = TEST_UTIL.createTable(tableName, FAMILY, 3)) {
 
         byte[] VALUEA = Bytes.toBytes("value-a");
@@ -6690,7 +6720,7 @@ public class TestFromClientSide {
 
   @Test
   public void testCellUtilTypeMethods() throws IOException {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     try (Table table = TEST_UTIL.createTable(tableName, FAMILY)) {
 
       final byte[] row = Bytes.toBytes("p");
@@ -6744,7 +6774,7 @@ public class TestFromClientSide {
 
   @Test(expected = DoNotRetryIOException.class)
   public void testCreateTableWithZeroRegionReplicas() throws Exception {
-    TableName tableName = TableName.valueOf(name.getMethodName());
+    TableName tableName = name.getTableName();
     TableDescriptor desc = TableDescriptorBuilder.newBuilder(tableName)
         .setColumnFamily(ColumnFamilyDescriptorBuilder.of(Bytes.toBytes("cf")))
         .setRegionReplication(0)
@@ -6755,7 +6785,7 @@ public class TestFromClientSide {
 
   @Test(expected = DoNotRetryIOException.class)
   public void testModifyTableWithZeroRegionReplicas() throws Exception {
-    TableName tableName = TableName.valueOf(name.getMethodName());
+    TableName tableName = name.getTableName();
     TableDescriptor desc = TableDescriptorBuilder.newBuilder(tableName)
         .setColumnFamily(ColumnFamilyDescriptorBuilder.of(Bytes.toBytes("cf")))
         .build();
@@ -6770,13 +6800,13 @@ public class TestFromClientSide {
 
   @Test(timeout = 60000)
   public void testModifyTableWithMemstoreData() throws Exception {
-    TableName tableName = TableName.valueOf(name.getMethodName());
+    TableName tableName = name.getTableName();
     createTableAndValidateTableSchemaModification(tableName, true);
   }
 
   @Test(timeout = 60000)
   public void testDeleteCFWithMemstoreData() throws Exception {
-    TableName tableName = TableName.valueOf(name.getMethodName());
+    TableName tableName = name.getTableName();
     createTableAndValidateTableSchemaModification(tableName, false);
   }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideWithCoprocessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideWithCoprocessor.java
index 37d0135..8845f9a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideWithCoprocessor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideWithCoprocessor.java
@@ -17,14 +17,16 @@
  */
 package org.apache.hadoop.hbase.client;
 
+import java.util.Arrays;
+import java.util.Collection;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint;
 import org.apache.hadoop.hbase.regionserver.NoOpScanPolicyObserver;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
-import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.experimental.categories.Category;
+import org.junit.runners.Parameterized;
 
 /**
  * Test all client operations with a coprocessor that just implements the default flush/compact/scan
@@ -32,13 +34,24 @@ import org.junit.experimental.categories.Category;
  */
 @Category({ LargeTests.class, ClientTests.class })
 public class TestFromClientSideWithCoprocessor extends TestFromClientSide {
-
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
     HBaseClassTestRule.forClass(TestFromClientSideWithCoprocessor.class);
 
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    initialize(MultiRowMutationEndpoint.class, NoOpScanPolicyObserver.class);
+  // Override the parameters from the parent class. We just want to run it for the default
+  // param combination.
+  @Parameterized.Parameters
+  public static Collection parameters() {
+    return Arrays.asList(new Object[][] {
+        { ZKConnectionRegistry.class, 1}
+    });
+  }
+
+  public TestFromClientSideWithCoprocessor(Class registry, int numHedgedReqs) throws Exception {
+    if (TEST_UTIL == null) {
+      // It is ok to initialize once because the test is parameterized for a single dimension.
+      initialize(registry, numHedgedReqs, NoOpScanPolicyObserver.class,
+          MultiRowMutationEndpoint.class);
+    }
   }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMasterRegistry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMasterRegistry.java
new file mode 100644
index 0000000..335f968
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMasterRegistry.java
@@ -0,0 +1,125 @@
+/*
+ * 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.HConstants.META_REPLICAS_NUM;
+import static org.junit.Assert.assertEquals;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.StartMiniClusterOption;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ MediumTests.class, ClientTests.class })
+public class TestMasterRegistry {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestMasterRegistry.class);
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    TEST_UTIL.getConfiguration().setInt(META_REPLICAS_NUM, 3);
+    StartMiniClusterOption.Builder builder = StartMiniClusterOption.builder();
+    builder.numMasters(3).numRegionServers(3);
+    TEST_UTIL.startMiniCluster(builder.build());
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  /**
+   * Generates a string of dummy master addresses in host:port format. Every other hostname won't
+   * have a port number.
+   */
+  private static String generateDummyMastersList(int size) {
+    List<String> masters = new ArrayList<>();
+    for (int i = 0; i < size; i++) {
+      masters.add(" localhost" + (i % 2 == 0 ? ":" + (1000 + i) : ""));
+    }
+    return String.join(",", masters);
+  }
+
+  /**
+   * Makes sure the master registry parses the master end points in the configuration correctly.
+   */
+  @Test public void testMasterAddressParsing() {
+    Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
+    int numMasters = 10;
+    conf.set(HConstants.MASTER_ADDRS_KEY, generateDummyMastersList(numMasters));
+    try (MasterRegistry registry = new MasterRegistry(conf)) {
+      List<ServerName> parsedMasters = new ArrayList<>(registry.getParsedMasterServers());
+      // Half of them would be without a port, duplicates are removed.
+      assertEquals(numMasters/2 + 1, parsedMasters.size());
+      // Sort in the increasing order of port numbers.
+      Collections.sort(parsedMasters, Comparator.comparingInt(ServerName::getPort));
+      for (int i = 0; i < parsedMasters.size(); i++) {
+        ServerName sn = parsedMasters.get(i);
+        assertEquals("localhost", sn.getHostname());
+        if (i == parsedMasters.size() - 1) {
+          // Last entry should be the one with default port.
+          assertEquals(HConstants.DEFAULT_MASTER_PORT, sn.getPort());
+        } else {
+          assertEquals(1000 + (2 * i), sn.getPort());
+        }
+      }
+    }
+  }
+
+  @Test public void testRegistryRPCs() throws Exception {
+    Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
+    HMaster activeMaster = TEST_UTIL.getHBaseCluster().getMaster();
+    for (int numHedgedReqs = 1; numHedgedReqs <=3; numHedgedReqs++) {
+      if (numHedgedReqs == 1) {
+        conf.setBoolean(HConstants.MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY, false);
+      } else {
+        conf.setBoolean(HConstants.MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY, true);
+      }
+      conf.setInt(HConstants.HBASE_RPCS_HEDGED_REQS_FANOUT_KEY, numHedgedReqs);
+      try (MasterRegistry registry = new MasterRegistry(conf)) {
+        assertEquals(registry.getClusterId().get(), activeMaster.getClusterId());
+        assertEquals(registry.getActiveMaster().get(), activeMaster.getServerName());
+        List<HRegionLocation> metaLocations =
+            Arrays.asList(registry.getMetaRegionLocations().get().getRegionLocations());
+        List<HRegionLocation> actualMetaLocations = activeMaster.getMetaRegionLocationCache()
+            .getMetaRegionLocations().get();
+        Collections.sort(metaLocations);
+        Collections.sort(actualMetaLocations);
+        assertEquals(actualMetaLocations, metaLocations);
+      }
+    }
+  }
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java
index c3da587..7388a90 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java
@@ -29,9 +29,10 @@ import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
-
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
 import java.util.List;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
@@ -44,35 +45,38 @@ import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.HTestConst;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
-import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.filter.FilterBase;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.StartMiniClusterOption;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.TableNameTestRule;
 import org.apache.hadoop.hbase.TableNotFoundException;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.filter.BinaryComparator;
 import org.apache.hadoop.hbase.filter.ColumnPrefixFilter;
 import org.apache.hadoop.hbase.filter.ColumnRangeFilter;
+import org.apache.hadoop.hbase.filter.FilterBase;
 import org.apache.hadoop.hbase.filter.QualifierFilter;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 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.EnvironmentEdgeManager;
-import org.junit.After;
 import org.junit.AfterClass;
-import org.junit.Before;
-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.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
 
 /**
- * A client-side test, mostly testing scanners with various parameters.
+ * A client-side test, mostly testing scanners with various parameters. Parameterized on different
+ * registry implementations.
  */
 @Category({MediumTests.class, ClientTests.class})
+@RunWith(Parameterized.class)
 public class TestScannersFromClientSide {
 
   @ClassRule
@@ -81,38 +85,80 @@ public class TestScannersFromClientSide {
 
   private static final Logger LOG = LoggerFactory.getLogger(TestScannersFromClientSide.class);
 
-  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static HBaseTestingUtility TEST_UTIL;
   private static byte [] ROW = Bytes.toBytes("testRow");
   private static byte [] FAMILY = Bytes.toBytes("testFamily");
   private static byte [] QUALIFIER = Bytes.toBytes("testQualifier");
   private static byte [] VALUE = Bytes.toBytes("testValue");
 
   @Rule
-  public TestName name = new TestName();
-
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    Configuration conf = TEST_UTIL.getConfiguration();
-    conf.setLong(HConstants.HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE_KEY, 10 * 1024 * 1024);
-    TEST_UTIL.startMiniCluster(3);
-  }
+  public TableNameTestRule name = new TableNameTestRule();
 
   @AfterClass
   public static void tearDownAfterClass() throws Exception {
-    TEST_UTIL.shutdownMiniCluster();
+    if (TEST_UTIL != null) {
+      TEST_UTIL.shutdownMiniCluster();
+    }
   }
 
-  @Before
-  public void setUp() throws Exception {
-    // Nothing to do.
+  @Parameterized.Parameters
+  public static Collection parameters() {
+    return Arrays.asList(new Object[][] {
+        { MasterRegistry.class, 1},
+        { MasterRegistry.class, 2},
+        { ZKConnectionRegistry.class, 1}
+    });
   }
 
   /**
-   * @throws java.lang.Exception
+   * JUnit does not provide an easy way to run a hook after each parameterized run. Without that
+   * there is no easy way to restart the test cluster after each parameterized run. Annotation
+   * BeforeParam does not work either because it runs before parameterization and hence does not
+   * have access to the test parameters (which is weird).
+   *
+   * This *hack* checks if the current instance of test cluster configuration has the passed
+   * parameterized configs. In such a case, we can just reuse the cluster for test and do not need
+   * to initialize from scratch. While this is a hack, it saves a ton of time for the full
+   * test and de-flakes it.
    */
-  @After
-  public void tearDown() throws Exception {
-    // Nothing to do.
+  private static boolean isSameParameterizedCluster(Class registryImpl, int numHedgedReqs) {
+    // initialize() is called for every unit test, however we only want to reset the cluster state
+    // at the end of every parameterized run.
+    if (TEST_UTIL == null) {
+      return false;
+    }
+    Configuration conf = TEST_UTIL.getConfiguration();
+    Class confClass = conf.getClass(
+        HConstants.CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY, ZKConnectionRegistry.class);
+    int hedgedReqConfig = conf.getInt(HConstants.HBASE_RPCS_HEDGED_REQS_FANOUT_KEY,
+        HConstants.HBASE_RPCS_HEDGED_REQS_FANOUT_DEFAULT);
+    return confClass.getName().equals(registryImpl.getName()) && numHedgedReqs == hedgedReqConfig;
+  }
+
+  public TestScannersFromClientSide(Class registryImpl, int numHedgedReqs) throws Exception {
+    if (isSameParameterizedCluster(registryImpl, numHedgedReqs)) {
+      return;
+    }
+    if (TEST_UTIL != null) {
+      // We reached the end of a parameterized run, clean up the cluster.
+      TEST_UTIL.shutdownMiniCluster();
+    }
+    TEST_UTIL = new HBaseTestingUtility();
+    Configuration conf = TEST_UTIL.getConfiguration();
+    conf.setLong(HConstants.HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE_KEY, 10 * 1024 * 1024);
+    conf.setClass(HConstants.CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY, registryImpl,
+        ConnectionRegistry.class);
+    if (numHedgedReqs == 1) {
+      conf.setBoolean(HConstants.MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY, false);
+    } else {
+      Preconditions.checkArgument(numHedgedReqs > 1);
+      conf.setBoolean(HConstants.MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY, true);
+    }
+    conf.setInt(HConstants.HBASE_RPCS_HEDGED_REQS_FANOUT_KEY, numHedgedReqs);
+    StartMiniClusterOption.Builder builder = StartMiniClusterOption.builder();
+    // Multiple masters needed only when hedged reads for master registry are enabled.
+    builder.numMasters(numHedgedReqs > 1 ? 3 : 1).numRegionServers(3);
+    TEST_UTIL.startMiniCluster(builder.build());
   }
 
   /**
@@ -120,7 +166,7 @@ public class TestScannersFromClientSide {
    */
   @Test
   public void testScanBatch() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     byte [][] QUALIFIERS = HTestConst.makeNAscii(QUALIFIER, 8);
 
     Table ht = TEST_UTIL.createTable(tableName, FAMILY);
@@ -190,7 +236,7 @@ public class TestScannersFromClientSide {
 
   @Test
   public void testMaxResultSizeIsSetToDefault() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     Table ht = TEST_UTIL.createTable(tableName, FAMILY);
 
     // The max result size we expect the scan to use by default.
@@ -259,7 +305,7 @@ public class TestScannersFromClientSide {
 
   @Test
   public void testSmallScan() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
 
     int numRows = 10;
     byte[][] ROWS = HTestConst.makeNAscii(ROW, numRows);
@@ -292,7 +338,8 @@ public class TestScannersFromClientSide {
   /**
    * Run through a variety of test configurations with a small scan
    */
-  private void testSmallScan(Table table, boolean reversed, int rows, int columns) throws Exception {
+  private void testSmallScan(
+      Table table, boolean reversed, int rows, int columns) throws Exception {
     Scan baseScan = new Scan();
     baseScan.setReversed(reversed);
     baseScan.setSmall(true);
@@ -334,7 +381,7 @@ public class TestScannersFromClientSide {
    */
   @Test
   public void testGetMaxResults() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     byte [][] FAMILIES = HTestConst.makeNAscii(FAMILY, 3);
     byte [][] QUALIFIERS = HTestConst.makeNAscii(QUALIFIER, 20);
 
@@ -408,8 +455,8 @@ public class TestScannersFromClientSide {
       kvListExp.add(new KeyValue(ROW, FAMILIES[1], QUALIFIERS[i], 1, VALUE));
     }
     for (int i=0; i < 2; i++) {
-        kvListExp.add(new KeyValue(ROW, FAMILIES[2], QUALIFIERS[i], 1, VALUE));
-      }
+      kvListExp.add(new KeyValue(ROW, FAMILIES[2], QUALIFIERS[i], 1, VALUE));
+    }
     for (int i=10; i < 20; i++) {
       kvListExp.add(new KeyValue(ROW, FAMILIES[2], QUALIFIERS[i], 1, VALUE));
     }
@@ -452,7 +499,7 @@ public class TestScannersFromClientSide {
    */
   @Test
   public void testScanMaxResults() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     byte [][] ROWS = HTestConst.makeNAscii(ROW, 2);
     byte [][] FAMILIES = HTestConst.makeNAscii(FAMILY, 3);
     byte [][] QUALIFIERS = HTestConst.makeNAscii(QUALIFIER, 10);
@@ -500,7 +547,7 @@ public class TestScannersFromClientSide {
    */
   @Test
   public void testGetRowOffset() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     byte [][] FAMILIES = HTestConst.makeNAscii(FAMILY, 3);
     byte [][] QUALIFIERS = HTestConst.makeNAscii(QUALIFIER, 20);
 
@@ -519,7 +566,9 @@ public class TestScannersFromClientSide {
       KeyValue kv = new KeyValue(ROW, FAMILIES[0], QUALIFIERS[i], 1, VALUE);
       put.add(kv);
       // skipping first two kvs
-      if (i < 2) continue;
+      if (i < 2) {
+        continue;
+      }
       kvListExp.add(kv);
     }
     ht.put(put);
@@ -590,7 +639,7 @@ public class TestScannersFromClientSide {
 
   @Test
   public void testScanRawDeleteFamilyVersion() throws Exception {
-    TableName tableName = TableName.valueOf(name.getMethodName());
+    TableName tableName = name.getTableName();
     TEST_UTIL.createTable(tableName, FAMILY);
     Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
     conf.set(RPC_CODEC_CONF_KEY, "");
@@ -618,7 +667,7 @@ public class TestScannersFromClientSide {
    */
   @Test
   public void testScanOnReopenedRegion() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     byte [][] QUALIFIERS = HTestConst.makeNAscii(QUALIFIER, 2);
 
     Table ht = TEST_UTIL.createTable(tableName, FAMILY);
@@ -693,8 +742,9 @@ public class TestScannersFromClientSide {
     LOG.info(msg);
     LOG.info("Expected count: " + expKvList.size());
     LOG.info("Actual count: " + result.size());
-    if (expKvList.isEmpty())
+    if (expKvList.isEmpty()) {
       return;
+    }
 
     int i = 0;
     for (Cell kv : result.rawCells()) {
@@ -715,7 +765,7 @@ public class TestScannersFromClientSide {
 
   @Test
   public void testReadExpiredDataForRawScan() throws IOException {
-    TableName tableName = TableName.valueOf(name.getMethodName());
+    TableName tableName = name.getTableName();
     long ts = System.currentTimeMillis() - 10000;
     byte[] value = Bytes.toBytes("expired");
     try (Table table = TEST_UTIL.createTable(tableName, FAMILY)) {
@@ -735,7 +785,7 @@ public class TestScannersFromClientSide {
 
   @Test
   public void testScanWithColumnsAndFilterAndVersion() throws IOException {
-    TableName tableName = TableName.valueOf(name.getMethodName());
+    TableName tableName = name.getTableName();
     try (Table table = TEST_UTIL.createTable(tableName, FAMILY, 4)) {
       for (int i = 0; i < 4; i++) {
         Put put = new Put(ROW);
@@ -757,7 +807,7 @@ public class TestScannersFromClientSide {
 
   @Test
   public void testScanWithSameStartRowStopRow() throws IOException {
-    TableName tableName = TableName.valueOf(name.getMethodName());
+    TableName tableName = name.getTableName();
     try (Table table = TEST_UTIL.createTable(tableName, FAMILY)) {
       table.put(new Put(ROW).addColumn(FAMILY, QUALIFIER, VALUE));
 
@@ -794,7 +844,7 @@ public class TestScannersFromClientSide {
 
   @Test
   public void testReverseScanWithFlush() throws Exception {
-    TableName tableName = TableName.valueOf(name.getMethodName());
+    TableName tableName = name.getTableName();
     final int BATCH_SIZE = 10;
     final int ROWS_TO_INSERT = 100;
     final byte[] LARGE_VALUE = generateHugeValue(128 * 1024);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java
index ac0d356..2797df3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java
@@ -30,22 +30,31 @@ import static org.mockito.Matchers.anyObject;
 import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.verify;
 import static org.mockito.internal.verification.VerificationModeFactory.times;
-
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.List;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ipc.RpcServer.BlockingServiceAndInterface;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.compress.GzipCodec;
+import org.apache.hadoop.util.StringUtils;
+import org.junit.Assume;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList;
+import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
+import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos;
 import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoRequestProto;
 import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoResponseProto;
 import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyRequestProto;
@@ -54,14 +63,6 @@ import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.PauseReq
 import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface;
 import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestRpcServiceProtos.TestProtobufRpcProto.Interface;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.io.compress.GzipCodec;
-import org.apache.hadoop.util.StringUtils;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList;
-import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 
 /**
  * Some basic ipc tests.
@@ -232,7 +233,6 @@ public abstract class AbstractTestIPC {
   /**
    * Tests that the RpcServer creates & dispatches CallRunner object to scheduler with non-null
    * remoteAddress set to its Call Object
-   * @throws ServiceException
    */
   @Test
   public void testRpcServerForNotNullRemoteAddressInCallObject()
@@ -363,6 +363,104 @@ public abstract class AbstractTestIPC {
     }
   }
 
+  /**
+   * Tests the various request fan out values using a simple RPC hedged across a mix of running and
+   * failing servers.
+   */
+  @Test
+  public void testHedgedAsyncEcho() throws Exception {
+    // Hedging is not supported for blocking connection types.
+    Assume.assumeFalse(this instanceof TestBlockingIPC);
+    List<RpcServer> rpcServers = new ArrayList<>();
+    List<InetSocketAddress> addresses = new ArrayList<>();
+    // Create a mix of running and failing servers.
+    final int numRunningServers = 5;
+    final int numFailingServers = 3;
+    final int numServers = numRunningServers + numFailingServers;
+    for (int i = 0; i < numRunningServers; i++) {
+      RpcServer rpcServer = createRpcServer(null, "testRpcServer" + i,
+          Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(
+          SERVICE, null)), new InetSocketAddress("localhost", 0), CONF,
+          new FifoRpcScheduler(CONF, 1));
+      rpcServer.start();
+      addresses.add(rpcServer.getListenerAddress());
+      rpcServers.add(rpcServer);
+    }
+    for (int i = 0; i < numFailingServers; i++) {
+      RpcServer rpcServer = createTestFailingRpcServer(null, "testFailingRpcServer" + i,
+          Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(
+          SERVICE, null)), new InetSocketAddress("localhost", 0), CONF,
+          new FifoRpcScheduler(CONF, 1));
+      rpcServer.start();
+      addresses.add(rpcServer.getListenerAddress());
+      rpcServers.add(rpcServer);
+    }
+    Configuration conf = HBaseConfiguration.create();
+    try (AbstractRpcClient<?> client = createRpcClient(conf)) {
+      // Try out various fan out values starting from 1 -> numServers.
+      for (int reqFanOut = 1; reqFanOut <= numServers; reqFanOut++) {
+        // Update the client's underlying conf, should be ok for the test.
+        LOG.debug("Testing with request fan out: " + reqFanOut);
+        conf.setInt(HConstants.HBASE_RPCS_HEDGED_REQS_FANOUT_KEY, reqFanOut);
+        Interface stub = newStub(client, addresses);
+        BlockingRpcCallback<EchoResponseProto> done = new BlockingRpcCallback<>();
+        stub.echo(new HBaseRpcControllerImpl(),
+            EchoRequestProto.newBuilder().setMessage("hello").build(), done);
+        TestProtos.EchoResponseProto responseProto = done.get();
+        assertNotNull(responseProto);
+        assertEquals("hello", responseProto.getMessage());
+        LOG.debug("Ended test with request fan out: " + reqFanOut);
+      }
+    } finally {
+      for (RpcServer rpcServer: rpcServers) {
+        rpcServer.stop();
+      }
+    }
+  }
+
+  @Test
+  public void testHedgedAsyncTimeouts() throws Exception {
+    // Hedging is not supported for blocking connection types.
+    Assume.assumeFalse(this instanceof TestBlockingIPC);
+    List<RpcServer> rpcServers = new ArrayList<>();
+    List<InetSocketAddress> addresses = new ArrayList<>();
+    final int numServers = 3;
+    for (int i = 0; i < numServers; i++) {
+      RpcServer rpcServer = createRpcServer(null, "testTimeoutRpcServer" + i,
+          Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(
+          SERVICE, null)), new InetSocketAddress("localhost", 0), CONF,
+          new FifoRpcScheduler(CONF, 1));
+      rpcServer.start();
+      addresses.add(rpcServer.getListenerAddress());
+      rpcServers.add(rpcServer);
+    }
+    Configuration conf = HBaseConfiguration.create();
+    int timeout = 100;
+    int pauseTime = 1000;
+    try (AbstractRpcClient<?> client = createRpcClient(conf)) {
+      // Try out various fan out values starting from 1 -> numServers.
+      for (int reqFanOut = 1; reqFanOut <= numServers; reqFanOut++) {
+        // Update the client's underlying conf, should be ok for the test.
+        LOG.debug("Testing with request fan out: " + reqFanOut);
+        conf.setInt(HConstants.HBASE_RPCS_HEDGED_REQS_FANOUT_KEY, reqFanOut);
+        Interface stub = newStub(client, addresses);
+        HBaseRpcController pcrc = new HBaseRpcControllerImpl();
+        pcrc.setCallTimeout(timeout);
+        BlockingRpcCallback<EmptyResponseProto> callback = new BlockingRpcCallback<>();
+        stub.pause(pcrc, PauseRequestProto.newBuilder().setMs(pauseTime).build(), callback);
+        assertNull(callback.get());
+        // Make sure the controller has the right exception propagated.
+        assertTrue(pcrc.getFailed() instanceof CallTimeoutException);
+        LOG.debug("Ended test with request fan out: " + reqFanOut);
+      }
+    } finally {
+      for (RpcServer rpcServer: rpcServers) {
+        rpcServer.stop();
+      }
+    }
+  }
+
+
   @Test
   public void testAsyncRemoteError() throws IOException {
     AbstractRpcClient<?> client = createRpcClient(CONF);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestProtobufRpcServiceImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestProtobufRpcServiceImpl.java
index d8a2d34..6adfa46 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestProtobufRpcServiceImpl.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestProtobufRpcServiceImpl.java
@@ -17,21 +17,23 @@
  */
 package org.apache.hadoop.hbase.ipc;
 
-import org.apache.hbase.thirdparty.com.google.protobuf.BlockingService;
-import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
-import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
-
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.util.ArrayList;
+import java.util.HashSet;
 import java.util.List;
-
+import java.util.Set;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.Threads;
 import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hbase.thirdparty.com.google.protobuf.BlockingService;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
+import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
 import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.AddrResponseProto;
 import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoRequestProto;
 import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EchoResponseProto;
@@ -41,8 +43,6 @@ import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.PauseReq
 import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestRpcServiceProtos.TestProtobufRpcProto;
 import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface;
 import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestRpcServiceProtos.TestProtobufRpcProto.Interface;
-import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.util.Threads;
 
 @InterfaceAudience.Private
 public class TestProtobufRpcServiceImpl implements BlockingInterface {
@@ -67,6 +67,17 @@ public class TestProtobufRpcServiceImpl implements BlockingInterface {
       User.getCurrent(), 0));
   }
 
+  public static Interface newStub(RpcClient client, List<InetSocketAddress> addrs)
+      throws IOException {
+    Set<ServerName> serverNames = new HashSet<>();
+    for (InetSocketAddress addr: addrs) {
+      serverNames.add(ServerName.valueOf(
+          addr.getHostName(), addr.getPort(), System.currentTimeMillis()));
+    }
+    return TestProtobufRpcProto.newStub(client.createHedgedRpcChannel(
+        serverNames, User.getCurrent(), 0));
+  }
+
   @Override
   public EmptyResponseProto ping(RpcController controller, EmptyRequestProto request)
       throws ServiceException {


[hbase] 05/09: HBASE-23648: Re-use underlying connection registry in RawAsyncHBaseAdmin (#994)

Posted by nd...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

ndimiduk pushed a commit to branch HBASE-18095/client-locate-meta-no-zookeeper
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit 7cc800d39bd48a16c8ab8be5969e534dd5d4e6f7
Author: Bharath Vissapragada <bh...@apache.org>
AuthorDate: Thu Jan 9 12:27:09 2020 -0800

    HBASE-23648: Re-use underlying connection registry in RawAsyncHBaseAdmin (#994)
    
    * HBASE-23648: Re-use underlying connection registry in RawAsyncHBaseAdmin
    
    No need to create and close a new registry on demand. Audited other
    usages of getRegistry() and the code looks fine.
    
    * Fix checkstyle issues in RawAsyncHBaseAdmin
---
 .../hadoop/hbase/client/RawAsyncHBaseAdmin.java    | 110 +++++++++------------
 1 file changed, 47 insertions(+), 63 deletions(-)

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 3e5bea3..69bd611 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
@@ -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
@@ -21,7 +21,6 @@ import static org.apache.hadoop.hbase.HConstants.HIGH_QOS;
 import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME;
 import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
 import static org.apache.hadoop.hbase.util.FutureUtils.unwrapCompletionException;
-
 import com.google.protobuf.Message;
 import com.google.protobuf.RpcChannel;
 import java.io.IOException;
@@ -46,7 +45,6 @@ import java.util.function.Supplier;
 import java.util.regex.Pattern;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
-import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.AsyncMetaTableAccessor;
 import org.apache.hadoop.hbase.CacheEvictionStats;
@@ -99,14 +97,12 @@ import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
 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.common.base.Preconditions;
 import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
 import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer;
 import org.apache.hbase.thirdparty.io.netty.util.Timeout;
 import org.apache.hbase.thirdparty.io.netty.util.TimerTask;
-
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos;
@@ -755,7 +751,8 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<Void> addColumnFamily(TableName tableName, ColumnFamilyDescriptor columnFamily) {
+  public CompletableFuture<Void> addColumnFamily(
+      TableName tableName, ColumnFamilyDescriptor columnFamily) {
     return this.<AddColumnRequest, AddColumnResponse> procedureCall(tableName,
       RequestConverter.buildAddColumnRequest(tableName, columnFamily, ng.getNonceGroup(),
         ng.newNonce()), (s, c, req, done) -> s.addColumn(c, req, done), (resp) -> resp.getProcId(),
@@ -809,10 +806,10 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
         .<NamespaceDescriptor> newMasterCaller()
         .action(
           (controller, stub) -> this
-              .<GetNamespaceDescriptorRequest, GetNamespaceDescriptorResponse, NamespaceDescriptor> call(
-                controller, stub, RequestConverter.buildGetNamespaceDescriptorRequest(name), (s, c,
-                    req, done) -> s.getNamespaceDescriptor(c, req, done), (resp) -> ProtobufUtil
-                    .toNamespaceDescriptor(resp.getNamespaceDescriptor()))).call();
+              .<GetNamespaceDescriptorRequest, GetNamespaceDescriptorResponse, NamespaceDescriptor>
+                  call(controller, stub, RequestConverter.buildGetNamespaceDescriptorRequest(name),
+                    (s, c, req, done) -> s.getNamespaceDescriptor(c, req, done), (resp)
+                      -> ProtobufUtil.toNamespaceDescriptor(resp.getNamespaceDescriptor()))).call();
   }
 
   @Override
@@ -830,13 +827,12 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   @Override
   public CompletableFuture<List<NamespaceDescriptor>> listNamespaceDescriptors() {
     return this
-        .<List<NamespaceDescriptor>> newMasterCaller()
-        .action(
-          (controller, stub) -> this
-              .<ListNamespaceDescriptorsRequest, ListNamespaceDescriptorsResponse, List<NamespaceDescriptor>> call(
-                controller, stub, ListNamespaceDescriptorsRequest.newBuilder().build(), (s, c, req,
-                    done) -> s.listNamespaceDescriptors(c, req, done), (resp) -> ProtobufUtil
-                    .toNamespaceDescriptorList(resp))).call();
+        .<List<NamespaceDescriptor>> newMasterCaller().action((controller, stub) -> this
+              .<ListNamespaceDescriptorsRequest, ListNamespaceDescriptorsResponse,
+                  List<NamespaceDescriptor>> call(controller, stub,
+                  ListNamespaceDescriptorsRequest.newBuilder().build(), (s, c, req, done) ->
+                      s.listNamespaceDescriptors(c, req, done),
+                    (resp) -> ProtobufUtil.toNamespaceDescriptorList(resp))).call();
   }
 
   @Override
@@ -1080,10 +1076,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   private CompletableFuture<List<HRegionLocation>> getTableHRegionLocations(TableName tableName) {
     if (TableName.META_TABLE_NAME.equals(tableName)) {
       CompletableFuture<List<HRegionLocation>> future = new CompletableFuture<>();
-      // For meta table, we use zk to fetch all locations.
-      ConnectionRegistry registry = ConnectionRegistryFactory.getRegistry(
-          connection.getConfiguration());
-      addListener(registry.getMetaRegionLocations(), (metaRegions, err) -> {
+      addListener(connection.registry.getMetaRegionLocations(), (metaRegions, err) -> {
         if (err != null) {
           future.completeExceptionally(err);
         } else if (metaRegions == null || metaRegions.isEmpty() ||
@@ -1092,8 +1085,6 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
         } else {
           future.complete(Collections.singletonList(metaRegions.getDefaultRegionLocation()));
         }
-        // close the registry.
-        IOUtils.closeQuietly(registry);
       });
       return future;
     } else {
@@ -1689,11 +1680,10 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   @Override
   public CompletableFuture<ReplicationPeerConfig> getReplicationPeerConfig(String peerId) {
     return this.<ReplicationPeerConfig> newMasterCaller().action((controller, stub) -> this
-      .<GetReplicationPeerConfigRequest, GetReplicationPeerConfigResponse, ReplicationPeerConfig> call(
-        controller, stub, RequestConverter.buildGetReplicationPeerConfigRequest(peerId),
-        (s, c, req, done) -> s.getReplicationPeerConfig(c, req, done),
-        (resp) -> ReplicationPeerConfigUtil.convert(resp.getPeerConfig())))
-      .call();
+      .<GetReplicationPeerConfigRequest, GetReplicationPeerConfigResponse, ReplicationPeerConfig>
+          call(controller, stub, RequestConverter.buildGetReplicationPeerConfigRequest(peerId),
+            (s, c, req, done) -> s.getReplicationPeerConfig(c, req, done),
+            (resp) -> ReplicationPeerConfigUtil.convert(resp.getPeerConfig()))).call();
   }
 
   @Override
@@ -1710,13 +1700,13 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   @Override
   public CompletableFuture<Void> transitReplicationPeerSyncReplicationState(String peerId,
       SyncReplicationState clusterState) {
-    return this
-      .<TransitReplicationPeerSyncReplicationStateRequest, TransitReplicationPeerSyncReplicationStateResponse> procedureCall(
+    return this.<TransitReplicationPeerSyncReplicationStateRequest,
+        TransitReplicationPeerSyncReplicationStateResponse> procedureCall(
         RequestConverter.buildTransitReplicationPeerSyncReplicationStateRequest(peerId,
           clusterState),
-        (s, c, req, done) -> s.transitReplicationPeerSyncReplicationState(c, req, done),
-        (resp) -> resp.getProcId(), new ReplicationProcedureBiConsumer(peerId,
-          () -> "TRANSIT_REPLICATION_PEER_SYNCHRONOUS_REPLICATION_STATE"));
+          (s, c, req, done) -> s.transitReplicationPeerSyncReplicationState(c, req, done),
+          (resp) -> resp.getProcId(), new ReplicationProcedureBiConsumer(peerId,
+            () -> "TRANSIT_REPLICATION_PEER_SYNCHRONOUS_REPLICATION_STATE"));
   }
 
   @Override
@@ -1786,11 +1776,8 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
     return this
         .<List<ReplicationPeerDescription>> newMasterCaller()
         .action(
-          (controller, stub) -> this
-              .<ListReplicationPeersRequest, ListReplicationPeersResponse, List<ReplicationPeerDescription>> call(
-                controller,
-                stub,
-                request,
+          (controller, stub) -> this.<ListReplicationPeersRequest, ListReplicationPeersResponse,
+              List<ReplicationPeerDescription>> call(controller, stub, request,
                 (s, c, req, done) -> s.listReplicationPeers(c, req, done),
                 (resp) -> resp.getPeerDescList().stream()
                     .map(ReplicationPeerConfigUtil::toReplicationPeerDescription)
@@ -2299,11 +2286,13 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<Void> decommissionRegionServers(List<ServerName> servers, boolean offload) {
+  public CompletableFuture<Void> decommissionRegionServers(
+      List<ServerName> servers, boolean offload) {
     return this.<Void> newMasterCaller()
         .action((controller, stub) -> this
           .<DecommissionRegionServersRequest, DecommissionRegionServersResponse, Void> call(
-            controller, stub, RequestConverter.buildDecommissionRegionServersRequest(servers, offload),
+            controller, stub,
+              RequestConverter.buildDecommissionRegionServersRequest(servers, offload),
             (s, c, req, done) -> s.decommissionRegionServers(c, req, done), resp -> null))
         .call();
   }
@@ -2325,11 +2314,11 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   public CompletableFuture<Void> recommissionRegionServer(ServerName server,
       List<byte[]> encodedRegionNames) {
     return this.<Void> newMasterCaller()
-        .action((controller, stub) -> this
-          .<RecommissionRegionServerRequest, RecommissionRegionServerResponse, Void> call(controller,
-            stub, RequestConverter.buildRecommissionRegionServerRequest(server, encodedRegionNames),
-            (s, c, req, done) -> s.recommissionRegionServer(c, req, done), resp -> null))
-        .call();
+        .action((controller, stub) ->
+            this.<RecommissionRegionServerRequest, RecommissionRegionServerResponse, Void> call(
+                controller, stub, RequestConverter.buildRecommissionRegionServerRequest(
+                    server, encodedRegionNames), (s, c, req, done) -> s.recommissionRegionServer(
+                        c, req, done), resp -> null)).call();
   }
 
   /**
@@ -2395,7 +2384,6 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
    * Get the region info for the passed region name. The region name may be a full region name or
    * encoded region name. If the region does not found, then it'll throw an UnknownRegionException
    * wrapped by a {@link CompletableFuture}
-   * @param regionNameOrEncodedRegionName
    * @return region info, wrapped by a {@link CompletableFuture}
    */
   private CompletableFuture<RegionInfo> getRegionInfo(byte[] regionNameOrEncodedRegionName) {
@@ -2886,10 +2874,11 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
         .<List<SecurityCapability>> newMasterCaller()
         .action(
           (controller, stub) -> this
-              .<SecurityCapabilitiesRequest, SecurityCapabilitiesResponse, List<SecurityCapability>> call(
-                controller, stub, SecurityCapabilitiesRequest.newBuilder().build(), (s, c, req,
-                    done) -> s.getSecurityCapabilities(c, req, done), (resp) -> ProtobufUtil
-                    .toSecurityCapabilityList(resp.getCapabilitiesList()))).call();
+              .<SecurityCapabilitiesRequest, SecurityCapabilitiesResponse, List<SecurityCapability>>
+                  call(controller, stub, SecurityCapabilitiesRequest.newBuilder().build(),
+                    (s, c, req, done) -> s.getSecurityCapabilities(c, req, done),
+                    (resp) -> ProtobufUtil.toSecurityCapabilityList(resp.getCapabilitiesList())))
+        .call();
   }
 
   @Override
@@ -3066,14 +3055,10 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
     MajorCompactionTimestampRequest request =
         MajorCompactionTimestampRequest.newBuilder()
             .setTableName(ProtobufUtil.toProtoTableName(tableName)).build();
-    return this
-        .<Optional<Long>> newMasterCaller()
-        .action(
-          (controller, stub) -> this
-              .<MajorCompactionTimestampRequest, MajorCompactionTimestampResponse, Optional<Long>> call(
-                controller, stub, request,
-                (s, c, req, done) -> s.getLastMajorCompactionTimestamp(c, req, done),
-                ProtobufUtil::toOptionalTimestamp)).call();
+    return this.<Optional<Long>> newMasterCaller().action((controller, stub) ->
+        this.<MajorCompactionTimestampRequest, MajorCompactionTimestampResponse, Optional<Long>>
+            call(controller, stub, request, (s, c, req, done) -> s.getLastMajorCompactionTimestamp(
+                c, req, done), ProtobufUtil::toOptionalTimestamp)).call();
   }
 
   @Override
@@ -3213,11 +3198,10 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   public CompletableFuture<Boolean> isBalancerEnabled() {
     return this
         .<Boolean> newMasterCaller()
-        .action(
-          (controller, stub) -> this.<IsBalancerEnabledRequest, IsBalancerEnabledResponse, Boolean> call(
-            controller, stub, RequestConverter.buildIsBalancerEnabledRequest(),
-            (s, c, req, done) -> s.isBalancerEnabled(c, req, done), (resp) -> resp.getEnabled()))
-        .call();
+        .action((controller, stub) ->
+              this.<IsBalancerEnabledRequest, IsBalancerEnabledResponse, Boolean> call(controller,
+            stub, RequestConverter.buildIsBalancerEnabledRequest(), (s, c, req, done)
+                  -> s.isBalancerEnabled(c, req, done), (resp) -> resp.getEnabled())).call();
   }
 
   @Override


[hbase] 01/09: HBASE-23275: Track active master's address in ActiveMasterManager (#812)

Posted by nd...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

ndimiduk pushed a commit to branch HBASE-18095/client-locate-meta-no-zookeeper
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit 084c61863167e7cab6225abc816493d239e1efc8
Author: Bharath Vissapragada <bh...@apache.org>
AuthorDate: Wed Nov 20 11:41:36 2019 -0800

    HBASE-23275: Track active master's address in ActiveMasterManager (#812)
    
    * HBASE-23275: Track active master's address in ActiveMasterManager
    
    Currently we just track whether an active master exists.
    It helps to also track the address of the active master in
    all the masters to help serve the client RPC requests to
    know which master is active.
    
    Signed-off-by: Nick Dimiduk <nd...@apache.org>
    Signed-off-by: Andrew Purtell <ap...@apache.org>
---
 .../hadoop/hbase/master/ActiveMasterManager.java   | 63 +++++++++++++++++-----
 .../org/apache/hadoop/hbase/master/HMaster.java    |  4 ++
 .../hbase/master/TestActiveMasterManager.java      | 10 ++++
 3 files changed, 64 insertions(+), 13 deletions(-)

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java
index 50798ed..99cab62 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java
@@ -1,4 +1,4 @@
-/**
+/*
  *
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
@@ -17,25 +17,24 @@
  * limitations under the License.
  */
 package org.apache.hadoop.hbase.master;
-
 import java.io.IOException;
+import java.util.Optional;
 import java.util.concurrent.atomic.AtomicBoolean;
-
-import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
-import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.ZNodeClearer;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
 import org.apache.hadoop.hbase.zookeeper.ZKListener;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
+import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 
 /**
  * Handles everything on master-side related to master election.
@@ -57,12 +56,18 @@ public class ActiveMasterManager extends ZKListener {
   final AtomicBoolean clusterHasActiveMaster = new AtomicBoolean(false);
   final AtomicBoolean clusterShutDown = new AtomicBoolean(false);
 
+  // This server's information.
   private final ServerName sn;
   private int infoPort;
   private final Server master;
 
+  // Active master's server name. Invalidated anytime active master changes (based on ZK
+  // notifications) and lazily fetched on-demand.
+  // ServerName is immutable, so we don't need heavy synchronization around it.
+  private volatile ServerName activeMasterServerName;
+
   /**
-   * @param watcher
+   * @param watcher ZK watcher
    * @param sn ServerName
    * @param master In an instance of a Master.
    */
@@ -107,6 +112,30 @@ public class ActiveMasterManager extends ZKListener {
   }
 
   /**
+   * Fetches the active master's ServerName from zookeeper.
+   */
+  private void fetchAndSetActiveMasterServerName() {
+    LOG.debug("Attempting to fetch active master sn from zk");
+    try {
+      activeMasterServerName = MasterAddressTracker.getMasterAddress(watcher);
+    } catch (IOException | KeeperException e) {
+      // Log and ignore for now and re-fetch later if needed.
+      LOG.error("Error fetching active master information", e);
+    }
+  }
+
+  public Optional<ServerName> getActiveMasterServerName() {
+    if (!clusterHasActiveMaster.get()) {
+      return Optional.empty();
+    }
+    if (activeMasterServerName == null) {
+      fetchAndSetActiveMasterServerName();
+    }
+    // It could still be null, but return whatever we have.
+    return Optional.ofNullable(activeMasterServerName);
+  }
+
+  /**
    * Handle a change in the master node.  Doesn't matter whether this was called
    * from a nodeCreated or nodeDeleted event because there are no guarantees
    * that the current state of the master node matches the event at the time of
@@ -134,6 +163,9 @@ public class ActiveMasterManager extends ZKListener {
           // Notify any thread waiting to become the active master
           clusterHasActiveMaster.notifyAll();
         }
+        // Reset the active master sn. Will be re-fetched later if needed.
+        // We don't want to make a synchronous RPC under a monitor.
+        activeMasterServerName = null;
       }
     } catch (KeeperException ke) {
       master.abort("Received an unexpected KeeperException, aborting", ke);
@@ -151,8 +183,8 @@ public class ActiveMasterManager extends ZKListener {
    * @param checkInterval the interval to check if the master is stopped
    * @param startupStatus the monitor status to track the progress
    * @return True if no issue becoming active master else false if another
-   * master was running or if some other problem (zookeeper, stop flag has been
-   * set on this Master)
+   *   master was running or if some other problem (zookeeper, stop flag has been
+   *   set on this Master)
    */
   boolean blockUntilBecomingActiveMaster(
       int checkInterval, MonitoredTask startupStatus) {
@@ -178,10 +210,14 @@ public class ActiveMasterManager extends ZKListener {
           // We are the master, return
           startupStatus.setStatus("Successfully registered as active master.");
           this.clusterHasActiveMaster.set(true);
+          activeMasterServerName = sn;
           LOG.info("Registered as active master=" + this.sn);
           return true;
         }
 
+        // Invalidate the active master name so that subsequent requests do not get any stale
+        // master information. Will be re-fetched if needed.
+        activeMasterServerName = null;
         // There is another active master running elsewhere or this is a restart
         // and the master ephemeral node has not expired yet.
         this.clusterHasActiveMaster.set(true);
@@ -208,7 +244,8 @@ public class ActiveMasterManager extends ZKListener {
             ZKUtil.deleteNode(this.watcher, this.watcher.getZNodePaths().masterAddressZNode);
 
             // We may have failed to delete the znode at the previous step, but
-            //  we delete the file anyway: a second attempt to delete the znode is likely to fail again.
+            //  we delete the file anyway: a second attempt to delete the znode is likely to fail
+            //  again.
             ZNodeClearer.deleteMyEphemeralNodeOnDisk();
           } else {
             msg = "Another master is the active master, " + currentMaster +
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 6864ce6..5f5d66d 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
@@ -3829,6 +3829,10 @@ public class HMaster extends HRegionServer implements MasterServices {
     return cachedClusterId.getFromCacheOrFetch();
   }
 
+  public Optional<ServerName> getActiveMaster() {
+    return activeMasterManager.getActiveMasterServerName();
+  }
+
   @Override
   public void runReplicationBarrierCleaner() {
     ReplicationBarrierCleaner rbc = this.replicationBarrierCleaner;
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 4649eea..84837f9 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
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hbase.master;
 
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
@@ -91,6 +92,7 @@ public class TestActiveMasterManager {
     ActiveMasterManager activeMasterManager =
       dummyMaster.getActiveMasterManager();
     assertFalse(activeMasterManager.clusterHasActiveMaster.get());
+    assertFalse(activeMasterManager.getActiveMasterServerName().isPresent());
 
     // First test becoming the active master uninterrupted
     MonitoredTask status = Mockito.mock(MonitoredTask.class);
@@ -99,6 +101,7 @@ public class TestActiveMasterManager {
     activeMasterManager.blockUntilBecomingActiveMaster(100, status);
     assertTrue(activeMasterManager.clusterHasActiveMaster.get());
     assertMaster(zk, master);
+    assertMaster(zk, activeMasterManager.getActiveMasterServerName().get());
 
     // Now pretend master restart
     DummyMaster secondDummyMaster = new DummyMaster(zk,master);
@@ -108,6 +111,8 @@ public class TestActiveMasterManager {
     activeMasterManager.blockUntilBecomingActiveMaster(100, status);
     assertTrue(activeMasterManager.clusterHasActiveMaster.get());
     assertMaster(zk, master);
+    assertMaster(zk, activeMasterManager.getActiveMasterServerName().get());
+    assertMaster(zk, secondActiveMasterManager.getActiveMasterServerName().get());
   }
 
   /**
@@ -135,6 +140,7 @@ public class TestActiveMasterManager {
     ActiveMasterManager activeMasterManager =
       ms1.getActiveMasterManager();
     assertFalse(activeMasterManager.clusterHasActiveMaster.get());
+    assertFalse(activeMasterManager.getActiveMasterServerName().isPresent());
 
     // First test becoming the active master uninterrupted
     ClusterStatusTracker clusterStatusTracker =
@@ -144,6 +150,7 @@ public class TestActiveMasterManager {
         Mockito.mock(MonitoredTask.class));
     assertTrue(activeMasterManager.clusterHasActiveMaster.get());
     assertMaster(zk, firstMasterAddress);
+    assertMaster(zk, activeMasterManager.getActiveMasterServerName().get());
 
     // New manager will now try to become the active master in another thread
     WaitToBeMasterThread t = new WaitToBeMasterThread(zk, secondMasterAddress);
@@ -161,6 +168,8 @@ public class TestActiveMasterManager {
     assertTrue(t.manager.clusterHasActiveMaster.get());
     // But secondary one should not be the active master
     assertFalse(t.isActiveMaster);
+    // Verify the active master ServerName is populated in standby master.
+    assertEquals(firstMasterAddress, t.manager.getActiveMasterServerName().get());
 
     // Close the first server and delete it's master node
     ms1.stop("stopping first server");
@@ -189,6 +198,7 @@ public class TestActiveMasterManager {
 
     assertTrue(t.manager.clusterHasActiveMaster.get());
     assertTrue(t.isActiveMaster);
+    assertEquals(secondMasterAddress, t.manager.getActiveMasterServerName().get());
 
     LOG.info("Deleting master node");
 


[hbase] 07/09: HBASE-23330: Fix delegation token fetch with MasterRegistry (#1084)

Posted by nd...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

ndimiduk pushed a commit to branch HBASE-18095/client-locate-meta-no-zookeeper
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit fcb201204385d4a48227138c49caa930224e75ea
Author: Bharath Vissapragada <bh...@apache.org>
AuthorDate: Fri Jan 24 18:04:21 2020 -0800

    HBASE-23330: Fix delegation token fetch with MasterRegistry (#1084)
    
    Signed-off-by: Andrew Purtell <ap...@apache.org>
---
 .../hadoop/hbase/client/AsyncConnectionImpl.java   |   10 +
 .../org/apache/hadoop/hbase/client/Connection.java |    5 +
 .../client/ConnectionOverAsyncConnection.java      |    5 +
 .../mapreduce/TestMultiTableInputFormatBase.java   |    6 +
 .../hbase/mapreduce/TestTableInputFormatBase.java  |    5 +
 .../hadoop/hbase/client/SharedConnection.java      |    5 +
 .../hadoop/hbase/security/token/TokenUtil.java     |   29 +-
 .../apache/hadoop/hbase/util/ConnectionCache.java  |   13 +
 .../hbase/thrift/ThriftHBaseServiceHandler.java    |    5 +
 .../hbase/thrift/generated/AlreadyExists.java      |    2 +-
 .../hbase/thrift/generated/BatchMutation.java      |    2 +-
 .../hbase/thrift/generated/ColumnDescriptor.java   |    2 +-
 .../hadoop/hbase/thrift/generated/Hbase.java       |  769 ++-
 .../hadoop/hbase/thrift/generated/IOError.java     |    2 +-
 .../hbase/thrift/generated/IllegalArgument.java    |    2 +-
 .../hadoop/hbase/thrift/generated/Mutation.java    |    2 +-
 .../hadoop/hbase/thrift/generated/TAppend.java     |    2 +-
 .../hadoop/hbase/thrift/generated/TCell.java       |    2 +-
 .../hadoop/hbase/thrift/generated/TColumn.java     |    2 +-
 .../hadoop/hbase/thrift/generated/TIncrement.java  |    2 +-
 .../hadoop/hbase/thrift/generated/TRegionInfo.java |    2 +-
 .../hadoop/hbase/thrift/generated/TRowResult.java  |    2 +-
 .../hadoop/hbase/thrift/generated/TScan.java       |    2 +-
 .../hbase/thrift/generated/TThriftServerType.java  |    2 +-
 .../hbase/thrift2/ThriftHBaseServiceHandler.java   |    5 +
 .../hbase/thrift2/client/ThriftConnection.java     |   23 +-
 .../hadoop/hbase/thrift2/generated/TAppend.java    |    2 +-
 .../hbase/thrift2/generated/TAuthorization.java    |    2 +-
 .../hbase/thrift2/generated/TBloomFilterType.java  |    2 +-
 .../hbase/thrift2/generated/TCellVisibility.java   |    2 +-
 .../hadoop/hbase/thrift2/generated/TColumn.java    |    2 +-
 .../thrift2/generated/TColumnFamilyDescriptor.java |    2 +-
 .../hbase/thrift2/generated/TColumnIncrement.java  |    2 +-
 .../hbase/thrift2/generated/TColumnValue.java      |    2 +-
 .../hbase/thrift2/generated/TCompareOperator.java  |    2 +-
 .../thrift2/generated/TCompressionAlgorithm.java   |    2 +-
 .../hbase/thrift2/generated/TConsistency.java      |    2 +-
 .../thrift2/generated/TDataBlockEncoding.java      |    2 +-
 .../hadoop/hbase/thrift2/generated/TDelete.java    |    2 +-
 .../hbase/thrift2/generated/TDeleteType.java       |    2 +-
 .../hbase/thrift2/generated/TDurability.java       |    2 +-
 .../hadoop/hbase/thrift2/generated/TGet.java       |    2 +-
 .../hbase/thrift2/generated/THBaseService.java     | 5163 +++++++++++---------
 .../hbase/thrift2/generated/THRegionInfo.java      |    2 +-
 .../hbase/thrift2/generated/THRegionLocation.java  |    2 +-
 .../hadoop/hbase/thrift2/generated/TIOError.java   |    2 +-
 .../hbase/thrift2/generated/TIllegalArgument.java  |    2 +-
 .../hadoop/hbase/thrift2/generated/TIncrement.java |    2 +-
 .../hbase/thrift2/generated/TKeepDeletedCells.java |    2 +-
 .../hadoop/hbase/thrift2/generated/TMutation.java  |    2 +-
 .../thrift2/generated/TNamespaceDescriptor.java    |    2 +-
 .../hadoop/hbase/thrift2/generated/TPut.java       |    2 +-
 .../hadoop/hbase/thrift2/generated/TReadType.java  |    2 +-
 .../hadoop/hbase/thrift2/generated/TResult.java    |    2 +-
 .../hbase/thrift2/generated/TRowMutations.java     |    2 +-
 .../hadoop/hbase/thrift2/generated/TScan.java      |    2 +-
 .../hbase/thrift2/generated/TServerName.java       |    2 +-
 .../hbase/thrift2/generated/TTableDescriptor.java  |    2 +-
 .../hadoop/hbase/thrift2/generated/TTableName.java |    2 +-
 .../hbase/thrift2/generated/TThriftServerType.java |    2 +-
 .../hadoop/hbase/thrift2/generated/TTimeRange.java |    2 +-
 .../org/apache/hadoop/hbase/thrift/Hbase.thrift    |    5 +
 .../org/apache/hadoop/hbase/thrift2/hbase.thrift   |    5 +
 .../hadoop/hbase/thrift2/TestThriftConnection.java |   19 +-
 64 files changed, 3890 insertions(+), 2278 deletions(-)

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 9d90249..bd39ac3 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
@@ -33,6 +33,7 @@ import java.util.Optional;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
@@ -278,6 +279,15 @@ class AsyncConnectionImpl implements AsyncConnection {
     }, stub -> true, "master stub");
   }
 
+  String getClusterId() {
+    try {
+      return registry.getClusterId().get();
+    } catch (InterruptedException | ExecutionException e) {
+      LOG.error("Error fetching cluster ID: ", e);
+    }
+    return null;
+  }
+
   void clearMasterStubCache(MasterService.Interface stub) {
     masterStub.compareAndSet(stub, null);
   }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Connection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Connection.java
index b88c40c..b638e72 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Connection.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Connection.java
@@ -205,6 +205,11 @@ public interface Connection extends Abortable, Closeable {
   AsyncConnection toAsyncConnection();
 
   /**
+   * @return the cluster ID unique to this HBase cluster.
+   */
+  String getClusterId();
+
+  /**
    * Retrieve an Hbck implementation to fix an HBase cluster.
    * The returned Hbck is not guaranteed to be thread-safe. A new instance should be created by
    * each thread. This is a lightweight operation. Pooling or caching of the returned Hbck instance
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionOverAsyncConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionOverAsyncConnection.java
index b61cef5..e50d308 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionOverAsyncConnection.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionOverAsyncConnection.java
@@ -205,6 +205,11 @@ class ConnectionOverAsyncConnection implements Connection {
   }
 
   @Override
+  public String getClusterId() {
+    return conn.getClusterId();
+  }
+
+  @Override
   public Hbck getHbck() throws IOException {
     return FutureUtils.get(conn.getHbck());
   }
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableInputFormatBase.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableInputFormatBase.java
index af97793..716d603 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableInputFormatBase.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableInputFormatBase.java
@@ -246,5 +246,11 @@ public class TestMultiTableInputFormatBase {
     public AsyncConnection toAsyncConnection() {
       return null;
     }
+
+    @Override
+    public String getClusterId() {
+      return null;
+    }
+
   }
 }
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatBase.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatBase.java
index 5fd5ccf..25b409e 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatBase.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatBase.java
@@ -296,5 +296,10 @@ public class TestTableInputFormatBase {
     public AsyncConnection toAsyncConnection() {
       throw new UnsupportedOperationException();
     }
+
+    @Override
+    public String getClusterId() {
+      return null;
+    }
   }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/SharedConnection.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/SharedConnection.java
index f189a2a..ae52df2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/SharedConnection.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/SharedConnection.java
@@ -105,4 +105,9 @@ public class SharedConnection implements Connection {
   public AsyncConnection toAsyncConnection() {
     return new SharedAsyncConnection(conn.toAsyncConnection());
   }
+
+  @Override
+  public String getClusterId() {
+    return conn.getClusterId();
+  }
 }
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 7cccea4..74a5d96 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
@@ -26,18 +26,14 @@ import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos;
 import org.apache.hadoop.hbase.security.User;
-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;
 import org.apache.hadoop.security.token.Token;
 import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-
 /**
  * Utility methods for obtaining authentication tokens.
  */
@@ -219,7 +215,7 @@ public class TokenUtil {
   public static void addTokenForJob(final Connection conn, final JobConf job, User user)
       throws IOException, InterruptedException {
 
-    Token<AuthenticationTokenIdentifier> token = getAuthToken(conn.getConfiguration(), user);
+    Token<AuthenticationTokenIdentifier> token = getAuthToken(conn, user);
     if (token == null) {
       token = ClientTokenUtil.obtainToken(conn, user);
     }
@@ -238,7 +234,7 @@ public class TokenUtil {
    */
   public static void addTokenForJob(final Connection conn, User user, Job job)
       throws IOException, InterruptedException {
-    Token<AuthenticationTokenIdentifier> token = getAuthToken(conn.getConfiguration(), user);
+    Token<AuthenticationTokenIdentifier> token = getAuthToken(conn, user);
     if (token == null) {
       token = ClientTokenUtil.obtainToken(conn, user);
     }
@@ -257,7 +253,7 @@ public class TokenUtil {
    */
   public static boolean addTokenIfMissing(Connection conn, User user)
       throws IOException, InterruptedException {
-    Token<AuthenticationTokenIdentifier> token = getAuthToken(conn.getConfiguration(), user);
+    Token<AuthenticationTokenIdentifier> token = getAuthToken(conn, user);
     if (token == null) {
       token = ClientTokenUtil.obtainToken(conn, user);
       user.getUGI().addToken(token.getService(), token);
@@ -270,19 +266,12 @@ public class TokenUtil {
    * Get the authentication token of the user for the cluster specified in the configuration
    * @return null if the user does not have the token, otherwise the auth token for the cluster.
    */
-  private static Token<AuthenticationTokenIdentifier> getAuthToken(Configuration conf, User user)
-      throws IOException, InterruptedException {
-    ZKWatcher zkw = new ZKWatcher(conf, "TokenUtil-getAuthToken", null);
-    try {
-      String clusterId = ZKClusterId.readClusterIdZNode(zkw);
-      if (clusterId == null) {
-        throw new IOException("Failed to get cluster ID");
-      }
-      return new AuthenticationTokenSelector().selectToken(new Text(clusterId), user.getTokens());
-    } catch (KeeperException e) {
-      throw new IOException(e);
-    } finally {
-      zkw.close();
+  private static Token<AuthenticationTokenIdentifier> getAuthToken(Connection conn, User user)
+      throws IOException {
+    final String clusterId = conn.getClusterId();
+    if (clusterId == null) {
+      throw new IOException("Failed to get cluster ID");
     }
+    return new AuthenticationTokenSelector().selectToken(new Text(clusterId), user.getTokens());
   }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ConnectionCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ConnectionCache.java
index 7b9f021..4559d78 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ConnectionCache.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ConnectionCache.java
@@ -202,6 +202,19 @@ public class ConnectionCache {
     return false;
   }
 
+  /**
+   * @return Cluster ID for the HBase cluster or null if there is an err making the connection.
+   */
+  public String getClusterId() {
+    try {
+      ConnectionInfo connInfo = getCurrentConnection();
+      return connInfo.connection.getClusterId();
+    } catch (IOException e) {
+      LOG.error("Error getting connection: ", e);
+    }
+    return null;
+  }
+
   class ConnectionInfo {
     final Connection connection;
     final String userName;
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
index e5ebb64..fe5e157 100644
--- 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
@@ -1272,6 +1272,11 @@ public class ThriftHBaseServiceHandler extends HBaseServiceHandler implements Hb
     return TThriftServerType.ONE;
   }
 
+  @Override
+  public String getClusterId() throws TException {
+    return connectionCache.getClusterId();
+  }
+
   private static IOError getIOError(Throwable throwable) {
     IOError error = new IOErrorWithCause(throwable);
     error.setMessage(Throwables.getStackTraceAsString(throwable));
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/AlreadyExists.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/AlreadyExists.java
index 340be18..9e74328 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/AlreadyExists.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/AlreadyExists.java
@@ -11,7 +11,7 @@ package org.apache.hadoop.hbase.thrift.generated;
  * An AlreadyExists exceptions signals that a table with the specified
  * name already exists
  */
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-07")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-01-22")
 public class AlreadyExists extends org.apache.thrift.TException implements org.apache.thrift.TBase<AlreadyExists, AlreadyExists._Fields>, java.io.Serializable, Cloneable, Comparable<AlreadyExists> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("AlreadyExists");
 
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/BatchMutation.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/BatchMutation.java
index c195977..da2432c 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/BatchMutation.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/BatchMutation.java
@@ -10,7 +10,7 @@ package org.apache.hadoop.hbase.thrift.generated;
 /**
  * A BatchMutation object is used to apply a number of Mutations to a single row.
  */
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-07")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-01-22")
 public class BatchMutation implements org.apache.thrift.TBase<BatchMutation, BatchMutation._Fields>, java.io.Serializable, Cloneable, Comparable<BatchMutation> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("BatchMutation");
 
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/ColumnDescriptor.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/ColumnDescriptor.java
index 084222e..46282ab 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/ColumnDescriptor.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/ColumnDescriptor.java
@@ -12,7 +12,7 @@ package org.apache.hadoop.hbase.thrift.generated;
  * such as the number of versions, compression settings, etc. It is
  * used as input when creating a table or adding a column.
  */
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-07")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-01-22")
 public class ColumnDescriptor implements org.apache.thrift.TBase<ColumnDescriptor, ColumnDescriptor._Fields>, java.io.Serializable, Cloneable, Comparable<ColumnDescriptor> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ColumnDescriptor");
 
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/Hbase.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/Hbase.java
index 9dc0d3e..fbd7208 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/Hbase.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/Hbase.java
@@ -7,7 +7,7 @@
 package org.apache.hadoop.hbase.thrift.generated;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-07")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-01-22")
 public class Hbase {
 
   public interface Iface {
@@ -622,6 +622,11 @@ public class Hbase {
      */
     public TThriftServerType getThriftServerType() throws org.apache.thrift.TException;
 
+    /**
+     * Returns the cluster ID for this cluster.
+     */
+    public java.lang.String getClusterId() throws org.apache.thrift.TException;
+
   }
 
   public interface AsyncIface {
@@ -716,6 +721,8 @@ public class Hbase {
 
     public void getThriftServerType(org.apache.thrift.async.AsyncMethodCallback<TThriftServerType> resultHandler) throws org.apache.thrift.TException;
 
+    public void getClusterId(org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler) throws org.apache.thrift.TException;
+
   }
 
   public static class Client extends org.apache.thrift.TServiceClient implements Iface {
@@ -1976,6 +1983,28 @@ public class Hbase {
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getThriftServerType failed: unknown result");
     }
 
+    public java.lang.String getClusterId() throws org.apache.thrift.TException
+    {
+      send_getClusterId();
+      return recv_getClusterId();
+    }
+
+    public void send_getClusterId() throws org.apache.thrift.TException
+    {
+      getClusterId_args args = new getClusterId_args();
+      sendBase("getClusterId", args);
+    }
+
+    public java.lang.String recv_getClusterId() throws org.apache.thrift.TException
+    {
+      getClusterId_result result = new getClusterId_result();
+      receiveBase(result, "getClusterId");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getClusterId failed: unknown result");
+    }
+
   }
   public static class AsyncClient extends org.apache.thrift.async.TAsyncClient implements AsyncIface {
     public static class Factory implements org.apache.thrift.async.TAsyncClientFactory<AsyncClient> {
@@ -3701,6 +3730,35 @@ public class Hbase {
       }
     }
 
+    public void getClusterId(org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      getClusterId_call method_call = new getClusterId_call(resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class getClusterId_call extends org.apache.thrift.async.TAsyncMethodCall<java.lang.String> {
+      public getClusterId_call(org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getClusterId", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        getClusterId_args args = new getClusterId_args();
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public java.lang.String getResult() throws org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new java.lang.IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_getClusterId();
+      }
+    }
+
   }
 
   public static class Processor<I extends Iface> extends org.apache.thrift.TBaseProcessor<I> implements org.apache.thrift.TProcessor {
@@ -3759,6 +3817,7 @@ public class Hbase {
       processMap.put("append", new append());
       processMap.put("checkAndPut", new checkAndPut());
       processMap.put("getThriftServerType", new getThriftServerType());
+      processMap.put("getClusterId", new getClusterId());
       return processMap;
     }
 
@@ -5094,6 +5153,31 @@ public class Hbase {
       }
     }
 
+    public static class getClusterId<I extends Iface> extends org.apache.thrift.ProcessFunction<I, getClusterId_args> {
+      public getClusterId() {
+        super("getClusterId");
+      }
+
+      public getClusterId_args getEmptyArgsInstance() {
+        return new getClusterId_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      @Override
+      protected boolean rethrowUnhandledExceptions() {
+        return false;
+      }
+
+      public getClusterId_result getResult(I iface, getClusterId_args args) throws org.apache.thrift.TException {
+        getClusterId_result result = new getClusterId_result();
+        result.success = iface.getClusterId();
+        return result;
+      }
+    }
+
   }
 
   public static class AsyncProcessor<I extends AsyncIface> extends org.apache.thrift.TBaseAsyncProcessor<I> {
@@ -5152,6 +5236,7 @@ public class Hbase {
       processMap.put("append", new append());
       processMap.put("checkAndPut", new checkAndPut());
       processMap.put("getThriftServerType", new getThriftServerType());
+      processMap.put("getClusterId", new getClusterId());
       return processMap;
     }
 
@@ -8112,6 +8197,67 @@ public class Hbase {
       }
     }
 
+    public static class getClusterId<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getClusterId_args, java.lang.String> {
+      public getClusterId() {
+        super("getClusterId");
+      }
+
+      public getClusterId_args getEmptyArgsInstance() {
+        return new getClusterId_args();
+      }
+
+      public org.apache.thrift.async.AsyncMethodCallback<java.lang.String> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new org.apache.thrift.async.AsyncMethodCallback<java.lang.String>() { 
+          public void onComplete(java.lang.String o) {
+            getClusterId_result result = new getClusterId_result();
+            result.success = o;
+            try {
+              fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+            } catch (org.apache.thrift.transport.TTransportException e) {
+              _LOGGER.error("TTransportException writing to internal frame buffer", e);
+              fb.close();
+            } catch (java.lang.Exception e) {
+              _LOGGER.error("Exception writing to internal frame buffer", e);
+              onError(e);
+            }
+          }
+          public void onError(java.lang.Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TSerializable msg;
+            getClusterId_result result = new getClusterId_result();
+            if (e instanceof org.apache.thrift.transport.TTransportException) {
+              _LOGGER.error("TTransportException inside handler", e);
+              fb.close();
+              return;
+            } else if (e instanceof org.apache.thrift.TApplicationException) {
+              _LOGGER.error("TApplicationException inside handler", e);
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TApplicationException)e;
+            } else {
+              _LOGGER.error("Exception inside handler", e);
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+            } catch (java.lang.Exception ex) {
+              _LOGGER.error("Exception writing to internal frame buffer", ex);
+              fb.close();
+            }
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, getClusterId_args args, org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler) throws org.apache.thrift.TException {
+        iface.getClusterId(resultHandler);
+      }
+    }
+
   }
 
   public static class enableTable_args implements org.apache.thrift.TBase<enableTable_args, enableTable_args._Fields>, java.io.Serializable, Cloneable, Comparable<enableTable_args>   {
@@ -60584,4 +60730,625 @@ public class Hbase {
     }
   }
 
+  public static class getClusterId_args implements org.apache.thrift.TBase<getClusterId_args, getClusterId_args._Fields>, java.io.Serializable, Cloneable, Comparable<getClusterId_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getClusterId_args");
+
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getClusterId_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getClusterId_argsTupleSchemeFactory();
+
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+;
+
+      private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
+
+      static {
+        for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      @org.apache.thrift.annotation.Nullable
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      @org.apache.thrift.annotation.Nullable
+      public static _Fields findByName(java.lang.String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final java.lang.String _fieldName;
+
+      _Fields(short thriftId, java.lang.String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public java.lang.String getFieldName() {
+        return _fieldName;
+      }
+    }
+    public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getClusterId_args.class, metaDataMap);
+    }
+
+    public getClusterId_args() {
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public getClusterId_args(getClusterId_args other) {
+    }
+
+    public getClusterId_args deepCopy() {
+      return new getClusterId_args(this);
+    }
+
+    @Override
+    public void clear() {
+    }
+
+    public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+      switch (field) {
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof getClusterId_args)
+        return this.equals((getClusterId_args)that);
+      return false;
+    }
+
+    public boolean equals(getClusterId_args that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return true;
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(getClusterId_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      return 0;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      scheme(iprot).read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      scheme(oprot).write(oprot, this);
+    }
+
+    @Override
+    public java.lang.String toString() {
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("getClusterId_args(");
+      boolean first = true;
+
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class getClusterId_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getClusterId_argsStandardScheme getScheme() {
+        return new getClusterId_argsStandardScheme();
+      }
+    }
+
+    private static class getClusterId_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<getClusterId_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, getClusterId_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+
+        // check for required fields of primitive type, which can't be checked in the validate method
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, getClusterId_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class getClusterId_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getClusterId_argsTupleScheme getScheme() {
+        return new getClusterId_argsTupleScheme();
+      }
+    }
+
+    private static class getClusterId_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<getClusterId_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, getClusterId_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, getClusterId_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+      }
+    }
+
+    private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
+      return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
+    }
+  }
+
+  public static class getClusterId_result implements org.apache.thrift.TBase<getClusterId_result, getClusterId_result._Fields>, java.io.Serializable, Cloneable, Comparable<getClusterId_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getClusterId_result");
+
+    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRING, (short)0);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getClusterId_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getClusterId_resultTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable java.lang.String success; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      SUCCESS((short)0, "success");
+
+      private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
+
+      static {
+        for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      @org.apache.thrift.annotation.Nullable
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 0: // SUCCESS
+            return SUCCESS;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      @org.apache.thrift.annotation.Nullable
+      public static _Fields findByName(java.lang.String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final java.lang.String _fieldName;
+
+      _Fields(short thriftId, java.lang.String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public java.lang.String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getClusterId_result.class, metaDataMap);
+    }
+
+    public getClusterId_result() {
+    }
+
+    public getClusterId_result(
+      java.lang.String success)
+    {
+      this();
+      this.success = success;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public getClusterId_result(getClusterId_result other) {
+      if (other.isSetSuccess()) {
+        this.success = other.success;
+      }
+    }
+
+    public getClusterId_result deepCopy() {
+      return new getClusterId_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.success = null;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.lang.String getSuccess() {
+      return this.success;
+    }
+
+    public getClusterId_result setSuccess(@org.apache.thrift.annotation.Nullable java.lang.String success) {
+      this.success = success;
+      return this;
+    }
+
+    public void unsetSuccess() {
+      this.success = null;
+    }
+
+    /** Returns true if field success is set (has been assigned a value) and false otherwise */
+    public boolean isSetSuccess() {
+      return this.success != null;
+    }
+
+    public void setSuccessIsSet(boolean value) {
+      if (!value) {
+        this.success = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+      switch (field) {
+      case SUCCESS:
+        if (value == null) {
+          unsetSuccess();
+        } else {
+          setSuccess((java.lang.String)value);
+        }
+        break;
+
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return getSuccess();
+
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case SUCCESS:
+        return isSetSuccess();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof getClusterId_result)
+        return this.equals((getClusterId_result)that);
+      return false;
+    }
+
+    public boolean equals(getClusterId_result that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return true;
+
+      boolean this_present_success = true && this.isSetSuccess();
+      boolean that_present_success = true && that.isSetSuccess();
+      if (this_present_success || that_present_success) {
+        if (!(this_present_success && that_present_success))
+          return false;
+        if (!this.success.equals(that.success))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      hashCode = hashCode * 8191 + ((isSetSuccess()) ? 131071 : 524287);
+      if (isSetSuccess())
+        hashCode = hashCode * 8191 + success.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(getClusterId_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = java.lang.Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetSuccess()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      scheme(iprot).read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      scheme(oprot).write(oprot, this);
+      }
+
+    @Override
+    public java.lang.String toString() {
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("getClusterId_result(");
+      boolean first = true;
+
+      sb.append("success:");
+      if (this.success == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.success);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class getClusterId_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getClusterId_resultStandardScheme getScheme() {
+        return new getClusterId_resultStandardScheme();
+      }
+    }
+
+    private static class getClusterId_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<getClusterId_result> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, getClusterId_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 0: // SUCCESS
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.success = iprot.readString();
+                struct.setSuccessIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+
+        // check for required fields of primitive type, which can't be checked in the validate method
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, getClusterId_result struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.success != null) {
+          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+          oprot.writeString(struct.success);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class getClusterId_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getClusterId_resultTupleScheme getScheme() {
+        return new getClusterId_resultTupleScheme();
+      }
+    }
+
+    private static class getClusterId_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<getClusterId_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, getClusterId_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet optionals = new java.util.BitSet();
+        if (struct.isSetSuccess()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetSuccess()) {
+          oprot.writeString(struct.success);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, getClusterId_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          struct.success = iprot.readString();
+          struct.setSuccessIsSet(true);
+        }
+      }
+    }
+
+    private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
+      return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
+    }
+  }
+
 }
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/IOError.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/IOError.java
index 2f7a547..0141f10 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/IOError.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/IOError.java
@@ -12,7 +12,7 @@ package org.apache.hadoop.hbase.thrift.generated;
  * to the Hbase master or an Hbase region server.  Also used to return
  * more general Hbase error conditions.
  */
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-07")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-01-22")
 public class IOError extends org.apache.thrift.TException implements org.apache.thrift.TBase<IOError, IOError._Fields>, java.io.Serializable, Cloneable, Comparable<IOError> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("IOError");
 
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/IllegalArgument.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/IllegalArgument.java
index 7f55f7d..fef4ef8 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/IllegalArgument.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/IllegalArgument.java
@@ -11,7 +11,7 @@ package org.apache.hadoop.hbase.thrift.generated;
  * An IllegalArgument exception indicates an illegal or invalid
  * argument was passed into a procedure.
  */
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-07")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-01-22")
 public class IllegalArgument extends org.apache.thrift.TException implements org.apache.thrift.TBase<IllegalArgument, IllegalArgument._Fields>, java.io.Serializable, Cloneable, Comparable<IllegalArgument> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("IllegalArgument");
 
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/Mutation.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/Mutation.java
index 35b8feb..b485bff 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/Mutation.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/Mutation.java
@@ -10,7 +10,7 @@ package org.apache.hadoop.hbase.thrift.generated;
 /**
  * A Mutation object is used to either update or delete a column-value.
  */
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-07")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-01-22")
 public class Mutation implements org.apache.thrift.TBase<Mutation, Mutation._Fields>, java.io.Serializable, Cloneable, Comparable<Mutation> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Mutation");
 
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TAppend.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TAppend.java
index 7be1c44..3d5e05f 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TAppend.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TAppend.java
@@ -10,7 +10,7 @@ package org.apache.hadoop.hbase.thrift.generated;
 /**
  * An Append object is used to specify the parameters for performing the append operation.
  */
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-07")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-01-22")
 public class TAppend implements org.apache.thrift.TBase<TAppend, TAppend._Fields>, java.io.Serializable, Cloneable, Comparable<TAppend> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TAppend");
 
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TCell.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TCell.java
index 20e16e3..0118355 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TCell.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TCell.java
@@ -13,7 +13,7 @@ package org.apache.hadoop.hbase.thrift.generated;
  * the timestamp of a cell to a first-class value, making it easy to take
  * note of temporal data. Cell is used all the way from HStore up to HTable.
  */
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-07")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-01-22")
 public class TCell implements org.apache.thrift.TBase<TCell, TCell._Fields>, java.io.Serializable, Cloneable, Comparable<TCell> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCell");
 
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TColumn.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TColumn.java
index e49ffa5..0c5993d 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TColumn.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TColumn.java
@@ -10,7 +10,7 @@ package org.apache.hadoop.hbase.thrift.generated;
 /**
  * Holds column name and the cell.
  */
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-07")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-01-22")
 public class TColumn implements org.apache.thrift.TBase<TColumn, TColumn._Fields>, java.io.Serializable, Cloneable, Comparable<TColumn> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumn");
 
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TIncrement.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TIncrement.java
index 0f2e82d..df84b2c 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TIncrement.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TIncrement.java
@@ -11,7 +11,7 @@ package org.apache.hadoop.hbase.thrift.generated;
  * For increments that are not incrementColumnValue
  * equivalents.
  */
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-07")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-01-22")
 public class TIncrement implements org.apache.thrift.TBase<TIncrement, TIncrement._Fields>, java.io.Serializable, Cloneable, Comparable<TIncrement> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TIncrement");
 
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TRegionInfo.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TRegionInfo.java
index 3dac155..f6e6651 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TRegionInfo.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TRegionInfo.java
@@ -10,7 +10,7 @@ package org.apache.hadoop.hbase.thrift.generated;
 /**
  * A TRegionInfo contains information about an HTable region.
  */
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-07")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-01-22")
 public class TRegionInfo implements org.apache.thrift.TBase<TRegionInfo, TRegionInfo._Fields>, java.io.Serializable, Cloneable, Comparable<TRegionInfo> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TRegionInfo");
 
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TRowResult.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TRowResult.java
index ffabc96..4a28099 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TRowResult.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TRowResult.java
@@ -10,7 +10,7 @@ package org.apache.hadoop.hbase.thrift.generated;
 /**
  * Holds row name and then a map of columns to cells.
  */
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-07")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-01-22")
 public class TRowResult implements org.apache.thrift.TBase<TRowResult, TRowResult._Fields>, java.io.Serializable, Cloneable, Comparable<TRowResult> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TRowResult");
 
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TScan.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TScan.java
index 9509228..9186cbb 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TScan.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TScan.java
@@ -10,7 +10,7 @@ package org.apache.hadoop.hbase.thrift.generated;
 /**
  * A Scan object is used to specify scanner parameters when opening a scanner.
  */
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-07")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-01-22")
 public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, java.io.Serializable, Cloneable, Comparable<TScan> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TScan");
 
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TThriftServerType.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TThriftServerType.java
index b7a792f..f38eb6f 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TThriftServerType.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TThriftServerType.java
@@ -10,7 +10,7 @@ package org.apache.hadoop.hbase.thrift.generated;
 /**
  * Specify type of thrift server: thrift and thrift2
  */
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-07")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-01-22")
 public enum TThriftServerType implements org.apache.thrift.TEnum {
   ONE(1),
   TWO(2);
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 15da348..a263956 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
@@ -818,6 +818,11 @@ public class ThriftHBaseServiceHandler extends HBaseServiceHandler implements TH
   }
 
   @Override
+  public String getClusterId() throws TException {
+    return connectionCache.getClusterId();
+  }
+
+  @Override
   public List<TNamespaceDescriptor> listNamespaceDescriptors() throws TIOError, TException {
     try {
       NamespaceDescriptor[] descriptors = connectionCache.getAdmin().listNamespaceDescriptors();
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftConnection.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftConnection.java
index 4db8fd6..ff1a79d 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftConnection.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftConnection.java
@@ -54,6 +54,7 @@ import org.apache.http.client.utils.HttpClientUtils;
 import org.apache.http.impl.client.DefaultHttpRequestRetryHandler;
 import org.apache.http.impl.client.HttpClientBuilder;
 import org.apache.http.protocol.HttpContext;
+import org.apache.thrift.TException;
 import org.apache.thrift.protocol.TBinaryProtocol;
 import org.apache.thrift.protocol.TCompactProtocol;
 import org.apache.thrift.protocol.TProtocol;
@@ -63,11 +64,14 @@ import org.apache.thrift.transport.TSocket;
 import org.apache.thrift.transport.TTransport;
 import org.apache.thrift.transport.TTransportException;
 import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
 
 @InterfaceAudience.Private
 public class ThriftConnection implements Connection {
+  private static final Logger LOG = LoggerFactory.getLogger(ThriftConnection.class);
   private Configuration conf;
   private User user;
   // For HTTP protocol
@@ -80,7 +84,8 @@ public class ThriftConnection implements Connection {
   private boolean isFramed = false;
   private boolean isCompact = false;
 
-  private ThriftClientBuilder clientBuilder;
+  // TODO: We can rip out the ThriftClient piece of it rather than creating a new client every time.
+  ThriftClientBuilder clientBuilder;
 
   private int operationTimeout;
   private int connectTimeout;
@@ -145,10 +150,6 @@ public class ThriftConnection implements Connection {
     return connectTimeout;
   }
 
-  public ThriftClientBuilder getClientBuilder() {
-    return clientBuilder;
-  }
-
   /**
    * the default thrift client builder.
    * One can extend the ThriftClientBuilder to builder custom client, implement
@@ -334,7 +335,6 @@ public class ThriftConnection implements Connection {
         } catch (IOException ioE) {
           throw new RuntimeException(ioE);
         }
-
       }
     };
   }
@@ -373,4 +373,15 @@ public class ThriftConnection implements Connection {
   public AsyncConnection toAsyncConnection() {
     throw new NotImplementedException("toAsyncConnection not supported in ThriftTable");
   }
+
+  @Override
+  public String getClusterId() {
+    try {
+      Pair<THBaseService.Client, TTransport> client = clientBuilder.getClient();
+      return client.getFirst().getClusterId();
+    } catch (TException | IOException e) {
+      LOG.error("Error fetching cluster ID: ", e);
+    }
+    return null;
+  }
 }
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TAppend.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TAppend.java
index b519163..28e48a1 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TAppend.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TAppend.java
@@ -7,7 +7,7 @@
 package org.apache.hadoop.hbase.thrift2.generated;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-07")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-01-22")
 public class TAppend implements org.apache.thrift.TBase<TAppend, TAppend._Fields>, java.io.Serializable, Cloneable, Comparable<TAppend> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TAppend");
 
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TAuthorization.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TAuthorization.java
index e6cd9b2..60de5d9 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TAuthorization.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TAuthorization.java
@@ -7,7 +7,7 @@
 package org.apache.hadoop.hbase.thrift2.generated;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-07")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-01-22")
 public class TAuthorization implements org.apache.thrift.TBase<TAuthorization, TAuthorization._Fields>, java.io.Serializable, Cloneable, Comparable<TAuthorization> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TAuthorization");
 
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TBloomFilterType.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TBloomFilterType.java
index 5c1276f..5ef1d12 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TBloomFilterType.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TBloomFilterType.java
@@ -11,7 +11,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
  * Thrift wrapper around
  * org.apache.hadoop.hbase.regionserver.BloomType
  */
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-07")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-01-22")
 public enum TBloomFilterType implements org.apache.thrift.TEnum {
   /**
    * Bloomfilters disabled
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCellVisibility.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCellVisibility.java
index ff50aa4..3791b54 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCellVisibility.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCellVisibility.java
@@ -7,7 +7,7 @@
 package org.apache.hadoop.hbase.thrift2.generated;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-07")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-01-22")
 public class TCellVisibility implements org.apache.thrift.TBase<TCellVisibility, TCellVisibility._Fields>, java.io.Serializable, Cloneable, Comparable<TCellVisibility> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCellVisibility");
 
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumn.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumn.java
index e72d9f8..98278aa 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumn.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumn.java
@@ -12,7 +12,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
  * in a HBase table by column family and optionally
  * a column qualifier and timestamp
  */
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-07")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-01-22")
 public class TColumn implements org.apache.thrift.TBase<TColumn, TColumn._Fields>, java.io.Serializable, Cloneable, Comparable<TColumn> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumn");
 
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnFamilyDescriptor.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnFamilyDescriptor.java
index 949e8a7..737518f 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnFamilyDescriptor.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnFamilyDescriptor.java
@@ -11,7 +11,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
  * Thrift wrapper around
  * org.apache.hadoop.hbase.client.ColumnFamilyDescriptor
  */
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-07")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-01-22")
 public class TColumnFamilyDescriptor implements org.apache.thrift.TBase<TColumnFamilyDescriptor, TColumnFamilyDescriptor._Fields>, java.io.Serializable, Cloneable, Comparable<TColumnFamilyDescriptor> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumnFamilyDescriptor");
 
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnIncrement.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnIncrement.java
index 4d97503..fe20ef4 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnIncrement.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnIncrement.java
@@ -10,7 +10,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
 /**
  * Represents a single cell and the amount to increment it by
  */
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-07")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-01-22")
 public class TColumnIncrement implements org.apache.thrift.TBase<TColumnIncrement, TColumnIncrement._Fields>, java.io.Serializable, Cloneable, Comparable<TColumnIncrement> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumnIncrement");
 
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnValue.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnValue.java
index 5579780..cb4e6be 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnValue.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnValue.java
@@ -10,7 +10,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
 /**
  * Represents a single cell and its value.
  */
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-07")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-01-22")
 public class TColumnValue implements org.apache.thrift.TBase<TColumnValue, TColumnValue._Fields>, java.io.Serializable, Cloneable, Comparable<TColumnValue> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumnValue");
 
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCompareOperator.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCompareOperator.java
index 019174c..d15a6c5 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCompareOperator.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCompareOperator.java
@@ -11,7 +11,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
  * Thrift wrapper around
  * org.apache.hadoop.hbase.CompareOperator.
  */
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-07")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-01-22")
 public enum TCompareOperator implements org.apache.thrift.TEnum {
   LESS(0),
   LESS_OR_EQUAL(1),
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCompressionAlgorithm.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCompressionAlgorithm.java
index 842d7bd..b1a23c1 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCompressionAlgorithm.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCompressionAlgorithm.java
@@ -11,7 +11,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
  * Thrift wrapper around
  * org.apache.hadoop.hbase.io.compress.Algorithm
  */
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-07")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-01-22")
 public enum TCompressionAlgorithm implements org.apache.thrift.TEnum {
   LZO(0),
   GZ(1),
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TConsistency.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TConsistency.java
index cc5c0cc..0f8519b 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TConsistency.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TConsistency.java
@@ -12,7 +12,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
  *  - STRONG means reads only from primary region
  *  - TIMELINE means reads might return values from secondary region replicas
  */
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-07")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-01-22")
 public enum TConsistency implements org.apache.thrift.TEnum {
   STRONG(1),
   TIMELINE(2);
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDataBlockEncoding.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDataBlockEncoding.java
index 05e2146..80a434a 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDataBlockEncoding.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDataBlockEncoding.java
@@ -11,7 +11,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
  * Thrift wrapper around
  * org.apache.hadoop.hbase.io.encoding.DataBlockEncoding
  */
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-07")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-01-22")
 public enum TDataBlockEncoding implements org.apache.thrift.TEnum {
   /**
    * Disable data block encoding.
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDelete.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDelete.java
index c11b409..8b634cc 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDelete.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDelete.java
@@ -33,7 +33,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
  * by changing the durability. If you don't provide durability, it defaults to
  * column family's default setting for durability.
  */
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-07")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-01-22")
 public class TDelete implements org.apache.thrift.TBase<TDelete, TDelete._Fields>, java.io.Serializable, Cloneable, Comparable<TDelete> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TDelete");
 
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDeleteType.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDeleteType.java
index e14eda4..60b57c8 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDeleteType.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDeleteType.java
@@ -12,7 +12,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
  *  - DELETE_COLUMN means exactly one version will be removed,
  *  - DELETE_COLUMNS means previous versions will also be removed.
  */
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-07")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-01-22")
 public enum TDeleteType implements org.apache.thrift.TEnum {
   DELETE_COLUMN(0),
   DELETE_COLUMNS(1),
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDurability.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDurability.java
index 65f43f9..b5f54ed 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDurability.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDurability.java
@@ -14,7 +14,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
  *  - SYNC_WAL means write the Mutation to the WAL synchronously,
  *  - FSYNC_WAL means Write the Mutation to the WAL synchronously and force the entries to disk.
  */
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-07")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-01-22")
 public enum TDurability implements org.apache.thrift.TEnum {
   USE_DEFAULT(0),
   SKIP_WAL(1),
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TGet.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TGet.java
index 8b2d17f..f55f7c2 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TGet.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TGet.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
  * If you specify a time range and a timestamp the range is ignored.
  * Timestamps on TColumns are ignored.
  */
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-07")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-01-22")
 public class TGet implements org.apache.thrift.TBase<TGet, TGet._Fields>, java.io.Serializable, Cloneable, Comparable<TGet> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGet");
 
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THBaseService.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THBaseService.java
index 7a984d5..668cbcf 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THBaseService.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THBaseService.java
@@ -7,7 +7,7 @@
 package org.apache.hadoop.hbase.thrift2.generated;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-11-07")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2020-01-22")
 public class THBaseService {
 
   public interface Iface {
@@ -507,6 +507,11 @@ public class THBaseService {
      */
     public TThriftServerType getThriftServerType() throws org.apache.thrift.TException;
 
+    /**
+     * Returns the cluster ID for this cluster.
+     */
+    public java.lang.String getClusterId() throws org.apache.thrift.TException;
+
   }
 
   public interface AsyncIface {
@@ -605,6 +610,8 @@ public class THBaseService {
 
     public void getThriftServerType(org.apache.thrift.async.AsyncMethodCallback<TThriftServerType> resultHandler) throws org.apache.thrift.TException;
 
+    public void getClusterId(org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler) throws org.apache.thrift.TException;
+
   }
 
   public static class Client extends org.apache.thrift.TServiceClient implements Iface {
@@ -1839,6 +1846,28 @@ public class THBaseService {
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getThriftServerType failed: unknown result");
     }
 
+    public java.lang.String getClusterId() throws org.apache.thrift.TException
+    {
+      send_getClusterId();
+      return recv_getClusterId();
+    }
+
+    public void send_getClusterId() throws org.apache.thrift.TException
+    {
+      getClusterId_args args = new getClusterId_args();
+      sendBase("getClusterId", args);
+    }
+
+    public java.lang.String recv_getClusterId() throws org.apache.thrift.TException
+    {
+      getClusterId_result result = new getClusterId_result();
+      receiveBase(result, "getClusterId");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getClusterId failed: unknown result");
+    }
+
   }
   public static class AsyncClient extends org.apache.thrift.async.TAsyncClient implements AsyncIface {
     public static class Factory implements org.apache.thrift.async.TAsyncClientFactory<AsyncClient> {
@@ -3475,6 +3504,35 @@ public class THBaseService {
       }
     }
 
+    public void getClusterId(org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      getClusterId_call method_call = new getClusterId_call(resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class getClusterId_call extends org.apache.thrift.async.TAsyncMethodCall<java.lang.String> {
+      public getClusterId_call(org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getClusterId", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        getClusterId_args args = new getClusterId_args();
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public java.lang.String getResult() throws org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new java.lang.IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_getClusterId();
+      }
+    }
+
   }
 
   public static class Processor<I extends Iface> extends org.apache.thrift.TBaseProcessor<I> implements org.apache.thrift.TProcessor {
@@ -3535,6 +3593,7 @@ public class THBaseService {
       processMap.put("listNamespaceDescriptors", new listNamespaceDescriptors());
       processMap.put("listNamespaces", new listNamespaces());
       processMap.put("getThriftServerType", new getThriftServerType());
+      processMap.put("getClusterId", new getClusterId());
       return processMap;
     }
 
@@ -4911,6 +4970,31 @@ public class THBaseService {
       }
     }
 
+    public static class getClusterId<I extends Iface> extends org.apache.thrift.ProcessFunction<I, getClusterId_args> {
+      public getClusterId() {
+        super("getClusterId");
+      }
+
+      public getClusterId_args getEmptyArgsInstance() {
+        return new getClusterId_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      @Override
+      protected boolean rethrowUnhandledExceptions() {
+        return false;
+      }
+
+      public getClusterId_result getResult(I iface, getClusterId_args args) throws org.apache.thrift.TException {
+        getClusterId_result result = new getClusterId_result();
+        result.success = iface.getClusterId();
+        return result;
+      }
+    }
+
   }
 
   public static class AsyncProcessor<I extends AsyncIface> extends org.apache.thrift.TBaseAsyncProcessor<I> {
@@ -4971,6 +5055,7 @@ public class THBaseService {
       processMap.put("listNamespaceDescriptors", new listNamespaceDescriptors());
       processMap.put("listNamespaces", new listNamespaces());
       processMap.put("getThriftServerType", new getThriftServerType());
+      processMap.put("getClusterId", new getClusterId());
       return processMap;
     }
 
@@ -8026,6 +8111,67 @@ public class THBaseService {
       }
     }
 
+    public static class getClusterId<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getClusterId_args, java.lang.String> {
+      public getClusterId() {
+        super("getClusterId");
+      }
+
+      public getClusterId_args getEmptyArgsInstance() {
+        return new getClusterId_args();
+      }
+
+      public org.apache.thrift.async.AsyncMethodCallback<java.lang.String> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new org.apache.thrift.async.AsyncMethodCallback<java.lang.String>() { 
+          public void onComplete(java.lang.String o) {
+            getClusterId_result result = new getClusterId_result();
+            result.success = o;
+            try {
+              fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+            } catch (org.apache.thrift.transport.TTransportException e) {
+              _LOGGER.error("TTransportException writing to internal frame buffer", e);
+              fb.close();
+            } catch (java.lang.Exception e) {
+              _LOGGER.error("Exception writing to internal frame buffer", e);
+              onError(e);
+            }
+          }
+          public void onError(java.lang.Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TSerializable msg;
+            getClusterId_result result = new getClusterId_result();
+            if (e instanceof org.apache.thrift.transport.TTransportException) {
+              _LOGGER.error("TTransportException inside handler", e);
+              fb.close();
+              return;
+            } else if (e instanceof org.apache.thrift.TApplicationException) {
+              _LOGGER.error("TApplicationException inside handler", e);
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TApplicationException)e;
+            } else {
+              _LOGGER.error("Exception inside handler", e);
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+            } catch (java.lang.Exception ex) {
+              _LOGGER.error("Exception writing to internal frame buffer", ex);
+              fb.close();
+            }
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, getClusterId_args args, org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler) throws org.apache.thrift.TException {
+        iface.getClusterId(resultHandler);
+      }
+    }
+
   }
 
   public static class exists_args implements org.apache.thrift.TBase<exists_args, exists_args._Fields>, java.io.Serializable, Cloneable, Comparable<exists_args>   {
@@ -36537,13 +36683,759 @@ public class THBaseService {
       tmpMap.put(_Fields.IO, new org.apache.thrift.meta_data.FieldMetaData("io", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TIOError.class)));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(createTable_result.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(createTable_result.class, metaDataMap);
+    }
+
+    public createTable_result() {
+    }
+
+    public createTable_result(
+      TIOError io)
+    {
+      this();
+      this.io = io;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public createTable_result(createTable_result other) {
+      if (other.isSetIo()) {
+        this.io = new TIOError(other.io);
+      }
+    }
+
+    public createTable_result deepCopy() {
+      return new createTable_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.io = null;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public TIOError getIo() {
+      return this.io;
+    }
+
+    public createTable_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) {
+      this.io = io;
+      return this;
+    }
+
+    public void unsetIo() {
+      this.io = null;
+    }
+
+    /** Returns true if field io is set (has been assigned a value) and false otherwise */
+    public boolean isSetIo() {
+      return this.io != null;
+    }
+
+    public void setIoIsSet(boolean value) {
+      if (!value) {
+        this.io = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+      switch (field) {
+      case IO:
+        if (value == null) {
+          unsetIo();
+        } else {
+          setIo((TIOError)value);
+        }
+        break;
+
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case IO:
+        return getIo();
+
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case IO:
+        return isSetIo();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof createTable_result)
+        return this.equals((createTable_result)that);
+      return false;
+    }
+
+    public boolean equals(createTable_result that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return true;
+
+      boolean this_present_io = true && this.isSetIo();
+      boolean that_present_io = true && that.isSetIo();
+      if (this_present_io || that_present_io) {
+        if (!(this_present_io && that_present_io))
+          return false;
+        if (!this.io.equals(that.io))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      hashCode = hashCode * 8191 + ((isSetIo()) ? 131071 : 524287);
+      if (isSetIo())
+        hashCode = hashCode * 8191 + io.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(createTable_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = java.lang.Boolean.valueOf(isSetIo()).compareTo(other.isSetIo());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetIo()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      scheme(iprot).read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      scheme(oprot).write(oprot, this);
+      }
+
+    @Override
+    public java.lang.String toString() {
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("createTable_result(");
+      boolean first = true;
+
+      sb.append("io:");
+      if (this.io == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.io);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class createTable_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public createTable_resultStandardScheme getScheme() {
+        return new createTable_resultStandardScheme();
+      }
+    }
+
+    private static class createTable_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<createTable_result> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, createTable_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 1: // IO
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.io = new TIOError();
+                struct.io.read(iprot);
+                struct.setIoIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+
+        // check for required fields of primitive type, which can't be checked in the validate method
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, createTable_result struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.io != null) {
+          oprot.writeFieldBegin(IO_FIELD_DESC);
+          struct.io.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class createTable_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public createTable_resultTupleScheme getScheme() {
+        return new createTable_resultTupleScheme();
+      }
+    }
+
+    private static class createTable_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<createTable_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, createTable_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet optionals = new java.util.BitSet();
+        if (struct.isSetIo()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetIo()) {
+          struct.io.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, createTable_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          struct.io = new TIOError();
+          struct.io.read(iprot);
+          struct.setIoIsSet(true);
+        }
+      }
+    }
+
+    private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
+      return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
+    }
+  }
+
+  public static class deleteTable_args implements org.apache.thrift.TBase<deleteTable_args, deleteTable_args._Fields>, java.io.Serializable, Cloneable, Comparable<deleteTable_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("deleteTable_args");
+
+    private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new deleteTable_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new deleteTable_argsTupleSchemeFactory();
+
+    /**
+     * the tablename to delete
+     */
+    public @org.apache.thrift.annotation.Nullable TTableName tableName; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      /**
+       * the tablename to delete
+       */
+      TABLE_NAME((short)1, "tableName");
+
+      private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
+
+      static {
+        for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      @org.apache.thrift.annotation.Nullable
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 1: // TABLE_NAME
+            return TABLE_NAME;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      @org.apache.thrift.annotation.Nullable
+      public static _Fields findByName(java.lang.String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final java.lang.String _fieldName;
+
+      _Fields(short thriftId, java.lang.String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public java.lang.String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TTableName.class)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(deleteTable_args.class, metaDataMap);
+    }
+
+    public deleteTable_args() {
+    }
+
+    public deleteTable_args(
+      TTableName tableName)
+    {
+      this();
+      this.tableName = tableName;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public deleteTable_args(deleteTable_args other) {
+      if (other.isSetTableName()) {
+        this.tableName = new TTableName(other.tableName);
+      }
+    }
+
+    public deleteTable_args deepCopy() {
+      return new deleteTable_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.tableName = null;
+    }
+
+    /**
+     * the tablename to delete
+     */
+    @org.apache.thrift.annotation.Nullable
+    public TTableName getTableName() {
+      return this.tableName;
+    }
+
+    /**
+     * the tablename to delete
+     */
+    public deleteTable_args setTableName(@org.apache.thrift.annotation.Nullable TTableName tableName) {
+      this.tableName = tableName;
+      return this;
+    }
+
+    public void unsetTableName() {
+      this.tableName = null;
+    }
+
+    /** Returns true if field tableName is set (has been assigned a value) and false otherwise */
+    public boolean isSetTableName() {
+      return this.tableName != null;
+    }
+
+    public void setTableNameIsSet(boolean value) {
+      if (!value) {
+        this.tableName = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+      switch (field) {
+      case TABLE_NAME:
+        if (value == null) {
+          unsetTableName();
+        } else {
+          setTableName((TTableName)value);
+        }
+        break;
+
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case TABLE_NAME:
+        return getTableName();
+
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case TABLE_NAME:
+        return isSetTableName();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof deleteTable_args)
+        return this.equals((deleteTable_args)that);
+      return false;
+    }
+
+    public boolean equals(deleteTable_args that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return true;
+
+      boolean this_present_tableName = true && this.isSetTableName();
+      boolean that_present_tableName = true && that.isSetTableName();
+      if (this_present_tableName || that_present_tableName) {
+        if (!(this_present_tableName && that_present_tableName))
+          return false;
+        if (!this.tableName.equals(that.tableName))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      hashCode = hashCode * 8191 + ((isSetTableName()) ? 131071 : 524287);
+      if (isSetTableName())
+        hashCode = hashCode * 8191 + tableName.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(deleteTable_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = java.lang.Boolean.valueOf(isSetTableName()).compareTo(other.isSetTableName());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetTableName()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, other.tableName);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      scheme(iprot).read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      scheme(oprot).write(oprot, this);
+    }
+
+    @Override
+    public java.lang.String toString() {
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("deleteTable_args(");
+      boolean first = true;
+
+      sb.append("tableName:");
+      if (this.tableName == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.tableName);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      if (tableName == null) {
+        throw new org.apache.thrift.protocol.TProtocolException("Required field 'tableName' was not present! Struct: " + toString());
+      }
+      // check for sub-struct validity
+      if (tableName != null) {
+        tableName.validate();
+      }
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class deleteTable_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public deleteTable_argsStandardScheme getScheme() {
+        return new deleteTable_argsStandardScheme();
+      }
+    }
+
+    private static class deleteTable_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<deleteTable_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, deleteTable_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 1: // TABLE_NAME
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.tableName = new TTableName();
+                struct.tableName.read(iprot);
+                struct.setTableNameIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+
+        // check for required fields of primitive type, which can't be checked in the validate method
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, deleteTable_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.tableName != null) {
+          oprot.writeFieldBegin(TABLE_NAME_FIELD_DESC);
+          struct.tableName.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class deleteTable_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public deleteTable_argsTupleScheme getScheme() {
+        return new deleteTable_argsTupleScheme();
+      }
+    }
+
+    private static class deleteTable_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<deleteTable_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, deleteTable_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        struct.tableName.write(oprot);
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, deleteTable_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        struct.tableName = new TTableName();
+        struct.tableName.read(iprot);
+        struct.setTableNameIsSet(true);
+      }
+    }
+
+    private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
+      return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
+    }
+  }
+
+  public static class deleteTable_result implements org.apache.thrift.TBase<deleteTable_result, deleteTable_result._Fields>, java.io.Serializable, Cloneable, Comparable<deleteTable_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("deleteTable_result");
+
+    private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new deleteTable_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new deleteTable_resultTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable TIOError io; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      IO((short)1, "io");
+
+      private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
+
+      static {
+        for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      @org.apache.thrift.annotation.Nullable
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 1: // IO
+            return IO;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      @org.apache.thrift.annotation.Nullable
+      public static _Fields findByName(java.lang.String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final java.lang.String _fieldName;
+
+      _Fields(short thriftId, java.lang.String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public java.lang.String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.IO, new org.apache.thrift.meta_data.FieldMetaData("io", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TIOError.class)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(deleteTable_result.class, metaDataMap);
     }
 
-    public createTable_result() {
+    public deleteTable_result() {
     }
 
-    public createTable_result(
+    public deleteTable_result(
       TIOError io)
     {
       this();
@@ -36553,14 +37445,14 @@ public class THBaseService {
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public createTable_result(createTable_result other) {
+    public deleteTable_result(deleteTable_result other) {
       if (other.isSetIo()) {
         this.io = new TIOError(other.io);
       }
     }
 
-    public createTable_result deepCopy() {
-      return new createTable_result(this);
+    public deleteTable_result deepCopy() {
+      return new deleteTable_result(this);
     }
 
     @Override
@@ -36573,7 +37465,7 @@ public class THBaseService {
       return this.io;
     }
 
-    public createTable_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) {
+    public deleteTable_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) {
       this.io = io;
       return this;
     }
@@ -36633,12 +37525,12 @@ public class THBaseService {
     public boolean equals(java.lang.Object that) {
       if (that == null)
         return false;
-      if (that instanceof createTable_result)
-        return this.equals((createTable_result)that);
+      if (that instanceof deleteTable_result)
+        return this.equals((deleteTable_result)that);
       return false;
     }
 
-    public boolean equals(createTable_result that) {
+    public boolean equals(deleteTable_result that) {
       if (that == null)
         return false;
       if (this == that)
@@ -36668,7 +37560,7 @@ public class THBaseService {
     }
 
     @Override
-    public int compareTo(createTable_result other) {
+    public int compareTo(deleteTable_result other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -36703,7 +37595,7 @@ public class THBaseService {
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("createTable_result(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("deleteTable_result(");
       boolean first = true;
 
       sb.append("io:");
@@ -36738,15 +37630,15 @@ public class THBaseService {
       }
     }
 
-    private static class createTable_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public createTable_resultStandardScheme getScheme() {
-        return new createTable_resultStandardScheme();
+    private static class deleteTable_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public deleteTable_resultStandardScheme getScheme() {
+        return new deleteTable_resultStandardScheme();
       }
     }
 
-    private static class createTable_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<createTable_result> {
+    private static class deleteTable_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<deleteTable_result> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, createTable_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, deleteTable_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -36776,7 +37668,7 @@ public class THBaseService {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, createTable_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, deleteTable_result struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -36791,16 +37683,16 @@ public class THBaseService {
 
     }
 
-    private static class createTable_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public createTable_resultTupleScheme getScheme() {
-        return new createTable_resultTupleScheme();
+    private static class deleteTable_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public deleteTable_resultTupleScheme getScheme() {
+        return new deleteTable_resultTupleScheme();
       }
     }
 
-    private static class createTable_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<createTable_result> {
+    private static class deleteTable_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<deleteTable_result> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, createTable_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, deleteTable_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet optionals = new java.util.BitSet();
         if (struct.isSetIo()) {
@@ -36813,7 +37705,7 @@ public class THBaseService {
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, createTable_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, deleteTable_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
@@ -36829,25 +37721,34 @@ public class THBaseService {
     }
   }
 
-  public static class deleteTable_args implements org.apache.thrift.TBase<deleteTable_args, deleteTable_args._Fields>, java.io.Serializable, Cloneable, Comparable<deleteTable_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("deleteTable_args");
+  public static class truncateTable_args implements org.apache.thrift.TBase<truncateTable_args, truncateTable_args._Fields>, java.io.Serializable, Cloneable, Comparable<truncateTable_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("truncateTable_args");
 
     private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+    private static final org.apache.thrift.protocol.TField PRESERVE_SPLITS_FIELD_DESC = new org.apache.thrift.protocol.TField("preserveSplits", org.apache.thrift.protocol.TType.BOOL, (short)2);
 
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new deleteTable_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new deleteTable_argsTupleSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new truncateTable_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new truncateTable_argsTupleSchemeFactory();
 
     /**
-     * the tablename to delete
+     * the tablename to truncate
      */
     public @org.apache.thrift.annotation.Nullable TTableName tableName; // required
+    /**
+     * whether to  preserve previous splits
+     */
+    public boolean preserveSplits; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
       /**
-       * the tablename to delete
+       * the tablename to truncate
        */
-      TABLE_NAME((short)1, "tableName");
+      TABLE_NAME((short)1, "tableName"),
+      /**
+       * whether to  preserve previous splits
+       */
+      PRESERVE_SPLITS((short)2, "preserveSplits");
 
       private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
 
@@ -36865,6 +37766,8 @@ public class THBaseService {
         switch(fieldId) {
           case 1: // TABLE_NAME
             return TABLE_NAME;
+          case 2: // PRESERVE_SPLITS
+            return PRESERVE_SPLITS;
           default:
             return null;
         }
@@ -36906,45 +37809,56 @@ public class THBaseService {
     }
 
     // isset id assignments
+    private static final int __PRESERVESPLITS_ISSET_ID = 0;
+    private byte __isset_bitfield = 0;
     public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
       tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TTableName.class)));
+      tmpMap.put(_Fields.PRESERVE_SPLITS, new org.apache.thrift.meta_data.FieldMetaData("preserveSplits", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(deleteTable_args.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(truncateTable_args.class, metaDataMap);
     }
 
-    public deleteTable_args() {
+    public truncateTable_args() {
     }
 
-    public deleteTable_args(
-      TTableName tableName)
+    public truncateTable_args(
+      TTableName tableName,
+      boolean preserveSplits)
     {
       this();
       this.tableName = tableName;
+      this.preserveSplits = preserveSplits;
+      setPreserveSplitsIsSet(true);
     }
 
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public deleteTable_args(deleteTable_args other) {
+    public truncateTable_args(truncateTable_args other) {
+      __isset_bitfield = other.__isset_bitfield;
       if (other.isSetTableName()) {
         this.tableName = new TTableName(other.tableName);
       }
+      this.preserveSplits = other.preserveSplits;
     }
 
-    public deleteTable_args deepCopy() {
-      return new deleteTable_args(this);
+    public truncateTable_args deepCopy() {
+      return new truncateTable_args(this);
     }
 
     @Override
     public void clear() {
       this.tableName = null;
+      setPreserveSplitsIsSet(false);
+      this.preserveSplits = false;
     }
 
     /**
-     * the tablename to delete
+     * the tablename to truncate
      */
     @org.apache.thrift.annotation.Nullable
     public TTableName getTableName() {
@@ -36952,9 +37866,9 @@ public class THBaseService {
     }
 
     /**
-     * the tablename to delete
+     * the tablename to truncate
      */
-    public deleteTable_args setTableName(@org.apache.thrift.annotation.Nullable TTableName tableName) {
+    public truncateTable_args setTableName(@org.apache.thrift.annotation.Nullable TTableName tableName) {
       this.tableName = tableName;
       return this;
     }
@@ -36974,6 +37888,35 @@ public class THBaseService {
       }
     }
 
+    /**
+     * whether to  preserve previous splits
+     */
+    public boolean isPreserveSplits() {
+      return this.preserveSplits;
+    }
+
+    /**
+     * whether to  preserve previous splits
+     */
+    public truncateTable_args setPreserveSplits(boolean preserveSplits) {
+      this.preserveSplits = preserveSplits;
+      setPreserveSplitsIsSet(true);
+      return this;
+    }
+
+    public void unsetPreserveSplits() {
+      __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __PRESERVESPLITS_ISSET_ID);
+    }
+
+    /** Returns true if field preserveSplits is set (has been assigned a value) and false otherwise */
+    public boolean isSetPreserveSplits() {
+      return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __PRESERVESPLITS_ISSET_ID);
+    }
+
+    public void setPreserveSplitsIsSet(boolean value) {
+      __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __PRESERVESPLITS_ISSET_ID, value);
+    }
+
     public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
       switch (field) {
       case TABLE_NAME:
@@ -36984,6 +37927,14 @@ public class THBaseService {
         }
         break;
 
+      case PRESERVE_SPLITS:
+        if (value == null) {
+          unsetPreserveSplits();
+        } else {
+          setPreserveSplits((java.lang.Boolean)value);
+        }
+        break;
+
       }
     }
 
@@ -36993,6 +37944,9 @@ public class THBaseService {
       case TABLE_NAME:
         return getTableName();
 
+      case PRESERVE_SPLITS:
+        return isPreserveSplits();
+
       }
       throw new java.lang.IllegalStateException();
     }
@@ -37006,6 +37960,8 @@ public class THBaseService {
       switch (field) {
       case TABLE_NAME:
         return isSetTableName();
+      case PRESERVE_SPLITS:
+        return isSetPreserveSplits();
       }
       throw new java.lang.IllegalStateException();
     }
@@ -37014,12 +37970,12 @@ public class THBaseService {
     public boolean equals(java.lang.Object that) {
       if (that == null)
         return false;
-      if (that instanceof deleteTable_args)
-        return this.equals((deleteTable_args)that);
+      if (that instanceof truncateTable_args)
+        return this.equals((truncateTable_args)that);
       return false;
     }
 
-    public boolean equals(deleteTable_args that) {
+    public boolean equals(truncateTable_args that) {
       if (that == null)
         return false;
       if (this == that)
@@ -37034,6 +37990,15 @@ public class THBaseService {
           return false;
       }
 
+      boolean this_present_preserveSplits = true;
+      boolean that_present_preserveSplits = true;
+      if (this_present_preserveSplits || that_present_preserveSplits) {
+        if (!(this_present_preserveSplits && that_present_preserveSplits))
+          return false;
+        if (this.preserveSplits != that.preserveSplits)
+          return false;
+      }
+
       return true;
     }
 
@@ -37045,11 +38010,13 @@ public class THBaseService {
       if (isSetTableName())
         hashCode = hashCode * 8191 + tableName.hashCode();
 
+      hashCode = hashCode * 8191 + ((preserveSplits) ? 131071 : 524287);
+
       return hashCode;
     }
 
     @Override
-    public int compareTo(deleteTable_args other) {
+    public int compareTo(truncateTable_args other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -37066,6 +38033,16 @@ public class THBaseService {
           return lastComparison;
         }
       }
+      lastComparison = java.lang.Boolean.valueOf(isSetPreserveSplits()).compareTo(other.isSetPreserveSplits());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetPreserveSplits()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.preserveSplits, other.preserveSplits);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
       return 0;
     }
 
@@ -37084,7 +38061,7 @@ public class THBaseService {
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("deleteTable_args(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("truncateTable_args(");
       boolean first = true;
 
       sb.append("tableName:");
@@ -37094,6 +38071,10 @@ public class THBaseService {
         sb.append(this.tableName);
       }
       first = false;
+      if (!first) sb.append(", ");
+      sb.append("preserveSplits:");
+      sb.append(this.preserveSplits);
+      first = false;
       sb.append(")");
       return sb.toString();
     }
@@ -37103,6 +38084,7 @@ public class THBaseService {
       if (tableName == null) {
         throw new org.apache.thrift.protocol.TProtocolException("Required field 'tableName' was not present! Struct: " + toString());
       }
+      // alas, we cannot check 'preserveSplits' because it's a primitive and you chose the non-beans generator.
       // check for sub-struct validity
       if (tableName != null) {
         tableName.validate();
@@ -37119,21 +38101,23 @@ public class THBaseService {
 
     private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
       try {
+        // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+        __isset_bitfield = 0;
         read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
       } catch (org.apache.thrift.TException te) {
         throw new java.io.IOException(te);
       }
     }
 
-    private static class deleteTable_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public deleteTable_argsStandardScheme getScheme() {
-        return new deleteTable_argsStandardScheme();
+    private static class truncateTable_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public truncateTable_argsStandardScheme getScheme() {
+        return new truncateTable_argsStandardScheme();
       }
     }
 
-    private static class deleteTable_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<deleteTable_args> {
+    private static class truncateTable_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<truncateTable_args> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, deleteTable_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, truncateTable_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -37152,6 +38136,14 @@ public class THBaseService {
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
               break;
+            case 2: // PRESERVE_SPLITS
+              if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+                struct.preserveSplits = iprot.readBool();
+                struct.setPreserveSplitsIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
             default:
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
           }
@@ -37160,10 +38152,13 @@ public class THBaseService {
         iprot.readStructEnd();
 
         // check for required fields of primitive type, which can't be checked in the validate method
+        if (!struct.isSetPreserveSplits()) {
+          throw new org.apache.thrift.protocol.TProtocolException("Required field 'preserveSplits' was not found in serialized data! Struct: " + toString());
+        }
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, deleteTable_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, truncateTable_args struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -37172,32 +38167,38 @@ public class THBaseService {
           struct.tableName.write(oprot);
           oprot.writeFieldEnd();
         }
+        oprot.writeFieldBegin(PRESERVE_SPLITS_FIELD_DESC);
+        oprot.writeBool(struct.preserveSplits);
+        oprot.writeFieldEnd();
         oprot.writeFieldStop();
         oprot.writeStructEnd();
       }
 
     }
 
-    private static class deleteTable_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public deleteTable_argsTupleScheme getScheme() {
-        return new deleteTable_argsTupleScheme();
+    private static class truncateTable_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public truncateTable_argsTupleScheme getScheme() {
+        return new truncateTable_argsTupleScheme();
       }
     }
 
-    private static class deleteTable_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<deleteTable_args> {
+    private static class truncateTable_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<truncateTable_args> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, deleteTable_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, truncateTable_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         struct.tableName.write(oprot);
+        oprot.writeBool(struct.preserveSplits);
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, deleteTable_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, truncateTable_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         struct.tableName = new TTableName();
         struct.tableName.read(iprot);
         struct.setTableNameIsSet(true);
+        struct.preserveSplits = iprot.readBool();
+        struct.setPreserveSplitsIsSet(true);
       }
     }
 
@@ -37206,13 +38207,13 @@ public class THBaseService {
     }
   }
 
-  public static class deleteTable_result implements org.apache.thrift.TBase<deleteTable_result, deleteTable_result._Fields>, java.io.Serializable, Cloneable, Comparable<deleteTable_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("deleteTable_result");
+  public static class truncateTable_result implements org.apache.thrift.TBase<truncateTable_result, truncateTable_result._Fields>, java.io.Serializable, Cloneable, Comparable<truncateTable_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("truncateTable_result");
 
     private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new deleteTable_resultStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new deleteTable_resultTupleSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new truncateTable_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new truncateTable_resultTupleSchemeFactory();
 
     public @org.apache.thrift.annotation.Nullable TIOError io; // required
 
@@ -37283,13 +38284,13 @@ public class THBaseService {
       tmpMap.put(_Fields.IO, new org.apache.thrift.meta_data.FieldMetaData("io", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TIOError.class)));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(deleteTable_result.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(truncateTable_result.class, metaDataMap);
     }
 
-    public deleteTable_result() {
+    public truncateTable_result() {
     }
 
-    public deleteTable_result(
+    public truncateTable_result(
       TIOError io)
     {
       this();
@@ -37299,14 +38300,14 @@ public class THBaseService {
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public deleteTable_result(deleteTable_result other) {
+    public truncateTable_result(truncateTable_result other) {
       if (other.isSetIo()) {
         this.io = new TIOError(other.io);
       }
     }
 
-    public deleteTable_result deepCopy() {
-      return new deleteTable_result(this);
+    public truncateTable_result deepCopy() {
+      return new truncateTable_result(this);
     }
 
     @Override
@@ -37319,7 +38320,7 @@ public class THBaseService {
       return this.io;
     }
 
-    public deleteTable_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) {
+    public truncateTable_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) {
       this.io = io;
       return this;
     }
@@ -37379,12 +38380,12 @@ public class THBaseService {
     public boolean equals(java.lang.Object that) {
       if (that == null)
         return false;
-      if (that instanceof deleteTable_result)
-        return this.equals((deleteTable_result)that);
+      if (that instanceof truncateTable_result)
+        return this.equals((truncateTable_result)that);
       return false;
     }
 
-    public boolean equals(deleteTable_result that) {
+    public boolean equals(truncateTable_result that) {
       if (that == null)
         return false;
       if (this == that)
@@ -37414,7 +38415,7 @@ public class THBaseService {
     }
 
     @Override
-    public int compareTo(deleteTable_result other) {
+    public int compareTo(truncateTable_result other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -37449,7 +38450,7 @@ public class THBaseService {
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("deleteTable_result(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("truncateTable_result(");
       boolean first = true;
 
       sb.append("io:");
@@ -37484,15 +38485,15 @@ public class THBaseService {
       }
     }
 
-    private static class deleteTable_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public deleteTable_resultStandardScheme getScheme() {
-        return new deleteTable_resultStandardScheme();
+    private static class truncateTable_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public truncateTable_resultStandardScheme getScheme() {
+        return new truncateTable_resultStandardScheme();
       }
     }
 
-    private static class deleteTable_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<deleteTable_result> {
+    private static class truncateTable_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<truncateTable_result> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, deleteTable_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, truncateTable_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -37522,7 +38523,7 @@ public class THBaseService {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, deleteTable_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, truncateTable_result struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -37537,16 +38538,16 @@ public class THBaseService {
 
     }
 
-    private static class deleteTable_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public deleteTable_resultTupleScheme getScheme() {
-        return new deleteTable_resultTupleScheme();
+    private static class truncateTable_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public truncateTable_resultTupleScheme getScheme() {
+        return new truncateTable_resultTupleScheme();
       }
     }
 
-    private static class deleteTable_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<deleteTable_result> {
+    private static class truncateTable_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<truncateTable_result> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, deleteTable_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, truncateTable_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet optionals = new java.util.BitSet();
         if (struct.isSetIo()) {
@@ -37559,7 +38560,7 @@ public class THBaseService {
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, deleteTable_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, truncateTable_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
@@ -37575,34 +38576,25 @@ public class THBaseService {
     }
   }
 
-  public static class truncateTable_args implements org.apache.thrift.TBase<truncateTable_args, truncateTable_args._Fields>, java.io.Serializable, Cloneable, Comparable<truncateTable_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("truncateTable_args");
+  public static class enableTable_args implements org.apache.thrift.TBase<enableTable_args, enableTable_args._Fields>, java.io.Serializable, Cloneable, Comparable<enableTable_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("enableTable_args");
 
     private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRUCT, (short)1);
-    private static final org.apache.thrift.protocol.TField PRESERVE_SPLITS_FIELD_DESC = new org.apache.thrift.protocol.TField("preserveSplits", org.apache.thrift.protocol.TType.BOOL, (short)2);
 
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new truncateTable_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new truncateTable_argsTupleSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new enableTable_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new enableTable_argsTupleSchemeFactory();
 
     /**
-     * the tablename to truncate
+     * the tablename to enable
      */
     public @org.apache.thrift.annotation.Nullable TTableName tableName; // required
-    /**
-     * whether to  preserve previous splits
-     */
-    public boolean preserveSplits; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
       /**
-       * the tablename to truncate
-       */
-      TABLE_NAME((short)1, "tableName"),
-      /**
-       * whether to  preserve previous splits
+       * the tablename to enable
        */
-      PRESERVE_SPLITS((short)2, "preserveSplits");
+      TABLE_NAME((short)1, "tableName");
 
       private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
 
@@ -37620,8 +38612,6 @@ public class THBaseService {
         switch(fieldId) {
           case 1: // TABLE_NAME
             return TABLE_NAME;
-          case 2: // PRESERVE_SPLITS
-            return PRESERVE_SPLITS;
           default:
             return null;
         }
@@ -37663,56 +38653,45 @@ public class THBaseService {
     }
 
     // isset id assignments
-    private static final int __PRESERVESPLITS_ISSET_ID = 0;
-    private byte __isset_bitfield = 0;
     public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
       tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TTableName.class)));
-      tmpMap.put(_Fields.PRESERVE_SPLITS, new org.apache.thrift.meta_data.FieldMetaData("preserveSplits", org.apache.thrift.TFieldRequirementType.REQUIRED, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(truncateTable_args.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(enableTable_args.class, metaDataMap);
     }
 
-    public truncateTable_args() {
+    public enableTable_args() {
     }
 
-    public truncateTable_args(
-      TTableName tableName,
-      boolean preserveSplits)
+    public enableTable_args(
+      TTableName tableName)
     {
       this();
       this.tableName = tableName;
-      this.preserveSplits = preserveSplits;
-      setPreserveSplitsIsSet(true);
     }
 
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public truncateTable_args(truncateTable_args other) {
-      __isset_bitfield = other.__isset_bitfield;
+    public enableTable_args(enableTable_args other) {
       if (other.isSetTableName()) {
         this.tableName = new TTableName(other.tableName);
       }
-      this.preserveSplits = other.preserveSplits;
     }
 
-    public truncateTable_args deepCopy() {
-      return new truncateTable_args(this);
+    public enableTable_args deepCopy() {
+      return new enableTable_args(this);
     }
 
     @Override
     public void clear() {
       this.tableName = null;
-      setPreserveSplitsIsSet(false);
-      this.preserveSplits = false;
     }
 
     /**
-     * the tablename to truncate
+     * the tablename to enable
      */
     @org.apache.thrift.annotation.Nullable
     public TTableName getTableName() {
@@ -37720,9 +38699,9 @@ public class THBaseService {
     }
 
     /**
-     * the tablename to truncate
+     * the tablename to enable
      */
-    public truncateTable_args setTableName(@org.apache.thrift.annotation.Nullable TTableName tableName) {
+    public enableTable_args setTableName(@org.apache.thrift.annotation.Nullable TTableName tableName) {
       this.tableName = tableName;
       return this;
     }
@@ -37742,35 +38721,6 @@ public class THBaseService {
       }
     }
 
-    /**
-     * whether to  preserve previous splits
-     */
-    public boolean isPreserveSplits() {
-      return this.preserveSplits;
-    }
-
-    /**
-     * whether to  preserve previous splits
-     */
-    public truncateTable_args setPreserveSplits(boolean preserveSplits) {
-      this.preserveSplits = preserveSplits;
-      setPreserveSplitsIsSet(true);
-      return this;
-    }
-
-    public void unsetPreserveSplits() {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __PRESERVESPLITS_ISSET_ID);
-    }
-
-    /** Returns true if field preserveSplits is set (has been assigned a value) and false otherwise */
-    public boolean isSetPreserveSplits() {
-      return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __PRESERVESPLITS_ISSET_ID);
-    }
-
-    public void setPreserveSplitsIsSet(boolean value) {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __PRESERVESPLITS_ISSET_ID, value);
-    }
-
     public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
       switch (field) {
       case TABLE_NAME:
@@ -37781,14 +38731,6 @@ public class THBaseService {
         }
         break;
 
-      case PRESERVE_SPLITS:
-        if (value == null) {
-          unsetPreserveSplits();
-        } else {
-          setPreserveSplits((java.lang.Boolean)value);
-        }
-        break;
-
       }
     }
 
@@ -37798,9 +38740,6 @@ public class THBaseService {
       case TABLE_NAME:
         return getTableName();
 
-      case PRESERVE_SPLITS:
-        return isPreserveSplits();
-
       }
       throw new java.lang.IllegalStateException();
     }
@@ -37814,8 +38753,6 @@ public class THBaseService {
       switch (field) {
       case TABLE_NAME:
         return isSetTableName();
-      case PRESERVE_SPLITS:
-        return isSetPreserveSplits();
       }
       throw new java.lang.IllegalStateException();
     }
@@ -37824,12 +38761,12 @@ public class THBaseService {
     public boolean equals(java.lang.Object that) {
       if (that == null)
         return false;
-      if (that instanceof truncateTable_args)
-        return this.equals((truncateTable_args)that);
+      if (that instanceof enableTable_args)
+        return this.equals((enableTable_args)that);
       return false;
     }
 
-    public boolean equals(truncateTable_args that) {
+    public boolean equals(enableTable_args that) {
       if (that == null)
         return false;
       if (this == that)
@@ -37844,15 +38781,6 @@ public class THBaseService {
           return false;
       }
 
-      boolean this_present_preserveSplits = true;
-      boolean that_present_preserveSplits = true;
-      if (this_present_preserveSplits || that_present_preserveSplits) {
-        if (!(this_present_preserveSplits && that_present_preserveSplits))
-          return false;
-        if (this.preserveSplits != that.preserveSplits)
-          return false;
-      }
-
       return true;
     }
 
@@ -37864,13 +38792,11 @@ public class THBaseService {
       if (isSetTableName())
         hashCode = hashCode * 8191 + tableName.hashCode();
 
-      hashCode = hashCode * 8191 + ((preserveSplits) ? 131071 : 524287);
-
       return hashCode;
     }
 
     @Override
-    public int compareTo(truncateTable_args other) {
+    public int compareTo(enableTable_args other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -37887,16 +38813,6 @@ public class THBaseService {
           return lastComparison;
         }
       }
-      lastComparison = java.lang.Boolean.valueOf(isSetPreserveSplits()).compareTo(other.isSetPreserveSplits());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetPreserveSplits()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.preserveSplits, other.preserveSplits);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
       return 0;
     }
 
@@ -37915,7 +38831,7 @@ public class THBaseService {
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("truncateTable_args(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("enableTable_args(");
       boolean first = true;
 
       sb.append("tableName:");
@@ -37925,10 +38841,6 @@ public class THBaseService {
         sb.append(this.tableName);
       }
       first = false;
-      if (!first) sb.append(", ");
-      sb.append("preserveSplits:");
-      sb.append(this.preserveSplits);
-      first = false;
       sb.append(")");
       return sb.toString();
     }
@@ -37938,7 +38850,6 @@ public class THBaseService {
       if (tableName == null) {
         throw new org.apache.thrift.protocol.TProtocolException("Required field 'tableName' was not present! Struct: " + toString());
       }
-      // alas, we cannot check 'preserveSplits' because it's a primitive and you chose the non-beans generator.
       // check for sub-struct validity
       if (tableName != null) {
         tableName.validate();
@@ -37955,23 +38866,21 @@ public class THBaseService {
 
     private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
       try {
-        // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
-        __isset_bitfield = 0;
         read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
       } catch (org.apache.thrift.TException te) {
         throw new java.io.IOException(te);
       }
     }
 
-    private static class truncateTable_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public truncateTable_argsStandardScheme getScheme() {
-        return new truncateTable_argsStandardScheme();
+    private static class enableTable_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public enableTable_argsStandardScheme getScheme() {
+        return new enableTable_argsStandardScheme();
       }
     }
 
-    private static class truncateTable_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<truncateTable_args> {
+    private static class enableTable_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<enableTable_args> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, truncateTable_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, enableTable_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -37990,14 +38899,6 @@ public class THBaseService {
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
               break;
-            case 2: // PRESERVE_SPLITS
-              if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
-                struct.preserveSplits = iprot.readBool();
-                struct.setPreserveSplitsIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
             default:
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
           }
@@ -38006,13 +38907,10 @@ public class THBaseService {
         iprot.readStructEnd();
 
         // check for required fields of primitive type, which can't be checked in the validate method
-        if (!struct.isSetPreserveSplits()) {
-          throw new org.apache.thrift.protocol.TProtocolException("Required field 'preserveSplits' was not found in serialized data! Struct: " + toString());
-        }
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, truncateTable_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, enableTable_args struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -38021,38 +38919,32 @@ public class THBaseService {
           struct.tableName.write(oprot);
           oprot.writeFieldEnd();
         }
-        oprot.writeFieldBegin(PRESERVE_SPLITS_FIELD_DESC);
-        oprot.writeBool(struct.preserveSplits);
-        oprot.writeFieldEnd();
         oprot.writeFieldStop();
         oprot.writeStructEnd();
       }
 
     }
 
-    private static class truncateTable_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public truncateTable_argsTupleScheme getScheme() {
-        return new truncateTable_argsTupleScheme();
+    private static class enableTable_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public enableTable_argsTupleScheme getScheme() {
+        return new enableTable_argsTupleScheme();
       }
     }
 
-    private static class truncateTable_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<truncateTable_args> {
+    private static class enableTable_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<enableTable_args> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, truncateTable_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, enableTable_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         struct.tableName.write(oprot);
-        oprot.writeBool(struct.preserveSplits);
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, truncateTable_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, enableTable_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         struct.tableName = new TTableName();
         struct.tableName.read(iprot);
         struct.setTableNameIsSet(true);
-        struct.preserveSplits = iprot.readBool();
-        struct.setPreserveSplitsIsSet(true);
       }
     }
 
@@ -38061,13 +38953,13 @@ public class THBaseService {
     }
   }
 
-  public static class truncateTable_result implements org.apache.thrift.TBase<truncateTable_result, truncateTable_result._Fields>, java.io.Serializable, Cloneable, Comparable<truncateTable_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("truncateTable_result");
+  public static class enableTable_result implements org.apache.thrift.TBase<enableTable_result, enableTable_result._Fields>, java.io.Serializable, Cloneable, Comparable<enableTable_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("enableTable_result");
 
     private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new truncateTable_resultStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new truncateTable_resultTupleSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new enableTable_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new enableTable_resultTupleSchemeFactory();
 
     public @org.apache.thrift.annotation.Nullable TIOError io; // required
 
@@ -38138,13 +39030,13 @@ public class THBaseService {
       tmpMap.put(_Fields.IO, new org.apache.thrift.meta_data.FieldMetaData("io", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TIOError.class)));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(truncateTable_result.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(enableTable_result.class, metaDataMap);
     }
 
-    public truncateTable_result() {
+    public enableTable_result() {
     }
 
-    public truncateTable_result(
+    public enableTable_result(
       TIOError io)
     {
       this();
@@ -38154,14 +39046,14 @@ public class THBaseService {
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public truncateTable_result(truncateTable_result other) {
+    public enableTable_result(enableTable_result other) {
       if (other.isSetIo()) {
         this.io = new TIOError(other.io);
       }
     }
 
-    public truncateTable_result deepCopy() {
-      return new truncateTable_result(this);
+    public enableTable_result deepCopy() {
+      return new enableTable_result(this);
     }
 
     @Override
@@ -38174,7 +39066,7 @@ public class THBaseService {
       return this.io;
     }
 
-    public truncateTable_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) {
+    public enableTable_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) {
       this.io = io;
       return this;
     }
@@ -38234,12 +39126,12 @@ public class THBaseService {
     public boolean equals(java.lang.Object that) {
       if (that == null)
         return false;
-      if (that instanceof truncateTable_result)
-        return this.equals((truncateTable_result)that);
+      if (that instanceof enableTable_result)
+        return this.equals((enableTable_result)that);
       return false;
     }
 
-    public boolean equals(truncateTable_result that) {
+    public boolean equals(enableTable_result that) {
       if (that == null)
         return false;
       if (this == that)
@@ -38269,7 +39161,7 @@ public class THBaseService {
     }
 
     @Override
-    public int compareTo(truncateTable_result other) {
+    public int compareTo(enableTable_result other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -38304,7 +39196,7 @@ public class THBaseService {
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("truncateTable_result(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("enableTable_result(");
       boolean first = true;
 
       sb.append("io:");
@@ -38339,15 +39231,15 @@ public class THBaseService {
       }
     }
 
-    private static class truncateTable_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public truncateTable_resultStandardScheme getScheme() {
-        return new truncateTable_resultStandardScheme();
+    private static class enableTable_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public enableTable_resultStandardScheme getScheme() {
+        return new enableTable_resultStandardScheme();
       }
     }
 
-    private static class truncateTable_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<truncateTable_result> {
+    private static class enableTable_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<enableTable_result> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, truncateTable_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, enableTable_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -38377,7 +39269,7 @@ public class THBaseService {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, truncateTable_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, enableTable_result struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -38392,16 +39284,16 @@ public class THBaseService {
 
     }
 
-    private static class truncateTable_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public truncateTable_resultTupleScheme getScheme() {
-        return new truncateTable_resultTupleScheme();
+    private static class enableTable_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public enableTable_resultTupleScheme getScheme() {
+        return new enableTable_resultTupleScheme();
       }
     }
 
-    private static class truncateTable_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<truncateTable_result> {
+    private static class enableTable_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<enableTable_result> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, truncateTable_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, enableTable_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet optionals = new java.util.BitSet();
         if (struct.isSetIo()) {
@@ -38414,7 +39306,7 @@ public class THBaseService {
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, truncateTable_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, enableTable_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
@@ -38430,23 +39322,23 @@ public class THBaseService {
     }
   }
 
-  public static class enableTable_args implements org.apache.thrift.TBase<enableTable_args, enableTable_args._Fields>, java.io.Serializable, Cloneable, Comparable<enableTable_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("enableTable_args");
+  public static class disableTable_args implements org.apache.thrift.TBase<disableTable_args, disableTable_args._Fields>, java.io.Serializable, Cloneable, Comparable<disableTable_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("disableTable_args");
 
     private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new enableTable_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new enableTable_argsTupleSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new disableTable_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new disableTable_argsTupleSchemeFactory();
 
     /**
-     * the tablename to enable
+     * the tablename to disable
      */
     public @org.apache.thrift.annotation.Nullable TTableName tableName; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
       /**
-       * the tablename to enable
+       * the tablename to disable
        */
       TABLE_NAME((short)1, "tableName");
 
@@ -38513,13 +39405,13 @@ public class THBaseService {
       tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TTableName.class)));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(enableTable_args.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(disableTable_args.class, metaDataMap);
     }
 
-    public enableTable_args() {
+    public disableTable_args() {
     }
 
-    public enableTable_args(
+    public disableTable_args(
       TTableName tableName)
     {
       this();
@@ -38529,14 +39421,14 @@ public class THBaseService {
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public enableTable_args(enableTable_args other) {
+    public disableTable_args(disableTable_args other) {
       if (other.isSetTableName()) {
         this.tableName = new TTableName(other.tableName);
       }
     }
 
-    public enableTable_args deepCopy() {
-      return new enableTable_args(this);
+    public disableTable_args deepCopy() {
+      return new disableTable_args(this);
     }
 
     @Override
@@ -38545,7 +39437,7 @@ public class THBaseService {
     }
 
     /**
-     * the tablename to enable
+     * the tablename to disable
      */
     @org.apache.thrift.annotation.Nullable
     public TTableName getTableName() {
@@ -38553,9 +39445,9 @@ public class THBaseService {
     }
 
     /**
-     * the tablename to enable
+     * the tablename to disable
      */
-    public enableTable_args setTableName(@org.apache.thrift.annotation.Nullable TTableName tableName) {
+    public disableTable_args setTableName(@org.apache.thrift.annotation.Nullable TTableName tableName) {
       this.tableName = tableName;
       return this;
     }
@@ -38615,12 +39507,12 @@ public class THBaseService {
     public boolean equals(java.lang.Object that) {
       if (that == null)
         return false;
-      if (that instanceof enableTable_args)
-        return this.equals((enableTable_args)that);
+      if (that instanceof disableTable_args)
+        return this.equals((disableTable_args)that);
       return false;
     }
 
-    public boolean equals(enableTable_args that) {
+    public boolean equals(disableTable_args that) {
       if (that == null)
         return false;
       if (this == that)
@@ -38650,7 +39542,7 @@ public class THBaseService {
     }
 
     @Override
-    public int compareTo(enableTable_args other) {
+    public int compareTo(disableTable_args other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -38685,7 +39577,7 @@ public class THBaseService {
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("enableTable_args(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("disableTable_args(");
       boolean first = true;
 
       sb.append("tableName:");
@@ -38726,15 +39618,15 @@ public class THBaseService {
       }
     }
 
-    private static class enableTable_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public enableTable_argsStandardScheme getScheme() {
-        return new enableTable_argsStandardScheme();
+    private static class disableTable_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public disableTable_argsStandardScheme getScheme() {
+        return new disableTable_argsStandardScheme();
       }
     }
 
-    private static class enableTable_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<enableTable_args> {
+    private static class disableTable_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<disableTable_args> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, enableTable_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, disableTable_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -38764,7 +39656,7 @@ public class THBaseService {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, enableTable_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, disableTable_args struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -38779,22 +39671,22 @@ public class THBaseService {
 
     }
 
-    private static class enableTable_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public enableTable_argsTupleScheme getScheme() {
-        return new enableTable_argsTupleScheme();
+    private static class disableTable_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public disableTable_argsTupleScheme getScheme() {
+        return new disableTable_argsTupleScheme();
       }
     }
 
-    private static class enableTable_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<enableTable_args> {
+    private static class disableTable_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<disableTable_args> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, enableTable_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, disableTable_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         struct.tableName.write(oprot);
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, enableTable_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, disableTable_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         struct.tableName = new TTableName();
         struct.tableName.read(iprot);
@@ -38807,13 +39699,13 @@ public class THBaseService {
     }
   }
 
-  public static class enableTable_result implements org.apache.thrift.TBase<enableTable_result, enableTable_result._Fields>, java.io.Serializable, Cloneable, Comparable<enableTable_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("enableTable_result");
+  public static class disableTable_result implements org.apache.thrift.TBase<disableTable_result, disableTable_result._Fields>, java.io.Serializable, Cloneable, Comparable<disableTable_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("disableTable_result");
 
     private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new enableTable_resultStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new enableTable_resultTupleSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new disableTable_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new disableTable_resultTupleSchemeFactory();
 
     public @org.apache.thrift.annotation.Nullable TIOError io; // required
 
@@ -38884,13 +39776,13 @@ public class THBaseService {
       tmpMap.put(_Fields.IO, new org.apache.thrift.meta_data.FieldMetaData("io", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TIOError.class)));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(enableTable_result.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(disableTable_result.class, metaDataMap);
     }
 
-    public enableTable_result() {
+    public disableTable_result() {
     }
 
-    public enableTable_result(
+    public disableTable_result(
       TIOError io)
     {
       this();
@@ -38900,14 +39792,14 @@ public class THBaseService {
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public enableTable_result(enableTable_result other) {
+    public disableTable_result(disableTable_result other) {
       if (other.isSetIo()) {
         this.io = new TIOError(other.io);
       }
     }
 
-    public enableTable_result deepCopy() {
-      return new enableTable_result(this);
+    public disableTable_result deepCopy() {
+      return new disableTable_result(this);
     }
 
     @Override
@@ -38920,7 +39812,7 @@ public class THBaseService {
       return this.io;
     }
 
-    public enableTable_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) {
+    public disableTable_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) {
       this.io = io;
       return this;
     }
@@ -38980,12 +39872,12 @@ public class THBaseService {
     public boolean equals(java.lang.Object that) {
       if (that == null)
         return false;
-      if (that instanceof enableTable_result)
-        return this.equals((enableTable_result)that);
+      if (that instanceof disableTable_result)
+        return this.equals((disableTable_result)that);
       return false;
     }
 
-    public boolean equals(enableTable_result that) {
+    public boolean equals(disableTable_result that) {
       if (that == null)
         return false;
       if (this == that)
@@ -39015,7 +39907,7 @@ public class THBaseService {
     }
 
     @Override
-    public int compareTo(enableTable_result other) {
+    public int compareTo(disableTable_result other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -39050,7 +39942,7 @@ public class THBaseService {
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("enableTable_result(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("disableTable_result(");
       boolean first = true;
 
       sb.append("io:");
@@ -39085,15 +39977,15 @@ public class THBaseService {
       }
     }
 
-    private static class enableTable_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public enableTable_resultStandardScheme getScheme() {
-        return new enableTable_resultStandardScheme();
+    private static class disableTable_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public disableTable_resultStandardScheme getScheme() {
+        return new disableTable_resultStandardScheme();
       }
     }
 
-    private static class enableTable_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<enableTable_result> {
+    private static class disableTable_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<disableTable_result> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, enableTable_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, disableTable_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -39123,7 +40015,7 @@ public class THBaseService {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, enableTable_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, disableTable_result struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -39138,16 +40030,16 @@ public class THBaseService {
 
     }
 
-    private static class enableTable_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public enableTable_resultTupleScheme getScheme() {
-        return new enableTable_resultTupleScheme();
+    private static class disableTable_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public disableTable_resultTupleScheme getScheme() {
+        return new disableTable_resultTupleScheme();
       }
     }
 
-    private static class enableTable_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<enableTable_result> {
+    private static class disableTable_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<disableTable_result> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, enableTable_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, disableTable_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet optionals = new java.util.BitSet();
         if (struct.isSetIo()) {
@@ -39160,7 +40052,7 @@ public class THBaseService {
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, enableTable_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, disableTable_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
@@ -39176,23 +40068,23 @@ public class THBaseService {
     }
   }
 
-  public static class disableTable_args implements org.apache.thrift.TBase<disableTable_args, disableTable_args._Fields>, java.io.Serializable, Cloneable, Comparable<disableTable_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("disableTable_args");
+  public static class isTableEnabled_args implements org.apache.thrift.TBase<isTableEnabled_args, isTableEnabled_args._Fields>, java.io.Serializable, Cloneable, Comparable<isTableEnabled_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("isTableEnabled_args");
 
     private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new disableTable_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new disableTable_argsTupleSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new isTableEnabled_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new isTableEnabled_argsTupleSchemeFactory();
 
     /**
-     * the tablename to disable
+     * the tablename to check
      */
     public @org.apache.thrift.annotation.Nullable TTableName tableName; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
       /**
-       * the tablename to disable
+       * the tablename to check
        */
       TABLE_NAME((short)1, "tableName");
 
@@ -39259,13 +40151,13 @@ public class THBaseService {
       tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TTableName.class)));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(disableTable_args.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(isTableEnabled_args.class, metaDataMap);
     }
 
-    public disableTable_args() {
+    public isTableEnabled_args() {
     }
 
-    public disableTable_args(
+    public isTableEnabled_args(
       TTableName tableName)
     {
       this();
@@ -39275,14 +40167,14 @@ public class THBaseService {
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public disableTable_args(disableTable_args other) {
+    public isTableEnabled_args(isTableEnabled_args other) {
       if (other.isSetTableName()) {
         this.tableName = new TTableName(other.tableName);
       }
     }
 
-    public disableTable_args deepCopy() {
-      return new disableTable_args(this);
+    public isTableEnabled_args deepCopy() {
+      return new isTableEnabled_args(this);
     }
 
     @Override
@@ -39291,7 +40183,7 @@ public class THBaseService {
     }
 
     /**
-     * the tablename to disable
+     * the tablename to check
      */
     @org.apache.thrift.annotation.Nullable
     public TTableName getTableName() {
@@ -39299,9 +40191,9 @@ public class THBaseService {
     }
 
     /**
-     * the tablename to disable
+     * the tablename to check
      */
-    public disableTable_args setTableName(@org.apache.thrift.annotation.Nullable TTableName tableName) {
+    public isTableEnabled_args setTableName(@org.apache.thrift.annotation.Nullable TTableName tableName) {
       this.tableName = tableName;
       return this;
     }
@@ -39361,12 +40253,12 @@ public class THBaseService {
     public boolean equals(java.lang.Object that) {
       if (that == null)
         return false;
-      if (that instanceof disableTable_args)
-        return this.equals((disableTable_args)that);
+      if (that instanceof isTableEnabled_args)
+        return this.equals((isTableEnabled_args)that);
       return false;
     }
 
-    public boolean equals(disableTable_args that) {
+    public boolean equals(isTableEnabled_args that) {
       if (that == null)
         return false;
       if (this == that)
@@ -39396,7 +40288,7 @@ public class THBaseService {
     }
 
     @Override
-    public int compareTo(disableTable_args other) {
+    public int compareTo(isTableEnabled_args other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -39431,7 +40323,7 @@ public class THBaseService {
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("disableTable_args(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("isTableEnabled_args(");
       boolean first = true;
 
       sb.append("tableName:");
@@ -39472,15 +40364,15 @@ public class THBaseService {
       }
     }
 
-    private static class disableTable_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public disableTable_argsStandardScheme getScheme() {
-        return new disableTable_argsStandardScheme();
+    private static class isTableEnabled_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public isTableEnabled_argsStandardScheme getScheme() {
+        return new isTableEnabled_argsStandardScheme();
       }
     }
 
-    private static class disableTable_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<disableTable_args> {
+    private static class isTableEnabled_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<isTableEnabled_args> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, disableTable_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, isTableEnabled_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -39510,7 +40402,7 @@ public class THBaseService {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, disableTable_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, isTableEnabled_args struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -39525,22 +40417,22 @@ public class THBaseService {
 
     }
 
-    private static class disableTable_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public disableTable_argsTupleScheme getScheme() {
-        return new disableTable_argsTupleScheme();
+    private static class isTableEnabled_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public isTableEnabled_argsTupleScheme getScheme() {
+        return new isTableEnabled_argsTupleScheme();
       }
     }
 
-    private static class disableTable_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<disableTable_args> {
+    private static class isTableEnabled_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<isTableEnabled_args> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, disableTable_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, isTableEnabled_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         struct.tableName.write(oprot);
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, disableTable_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, isTableEnabled_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         struct.tableName = new TTableName();
         struct.tableName.read(iprot);
@@ -39553,18 +40445,21 @@ public class THBaseService {
     }
   }
 
-  public static class disableTable_result implements org.apache.thrift.TBase<disableTable_result, disableTable_result._Fields>, java.io.Serializable, Cloneable, Comparable<disableTable_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("disableTable_result");
+  public static class isTableEnabled_result implements org.apache.thrift.TBase<isTableEnabled_result, isTableEnabled_result._Fields>, java.io.Serializable, Cloneable, Comparable<isTableEnabled_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("isTableEnabled_result");
 
+    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.BOOL, (short)0);
     private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new disableTable_resultStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new disableTable_resultTupleSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new isTableEnabled_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new isTableEnabled_resultTupleSchemeFactory();
 
+    public boolean success; // required
     public @org.apache.thrift.annotation.Nullable TIOError io; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      SUCCESS((short)0, "success"),
       IO((short)1, "io");
 
       private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
@@ -39581,6 +40476,8 @@ public class THBaseService {
       @org.apache.thrift.annotation.Nullable
       public static _Fields findByThriftId(int fieldId) {
         switch(fieldId) {
+          case 0: // SUCCESS
+            return SUCCESS;
           case 1: // IO
             return IO;
           default:
@@ -39624,49 +40521,83 @@ public class THBaseService {
     }
 
     // isset id assignments
+    private static final int __SUCCESS_ISSET_ID = 0;
+    private byte __isset_bitfield = 0;
     public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
       tmpMap.put(_Fields.IO, new org.apache.thrift.meta_data.FieldMetaData("io", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TIOError.class)));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(disableTable_result.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(isTableEnabled_result.class, metaDataMap);
     }
 
-    public disableTable_result() {
+    public isTableEnabled_result() {
     }
 
-    public disableTable_result(
+    public isTableEnabled_result(
+      boolean success,
       TIOError io)
     {
       this();
+      this.success = success;
+      setSuccessIsSet(true);
       this.io = io;
     }
 
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public disableTable_result(disableTable_result other) {
+    public isTableEnabled_result(isTableEnabled_result other) {
+      __isset_bitfield = other.__isset_bitfield;
+      this.success = other.success;
       if (other.isSetIo()) {
         this.io = new TIOError(other.io);
       }
     }
 
-    public disableTable_result deepCopy() {
-      return new disableTable_result(this);
+    public isTableEnabled_result deepCopy() {
+      return new isTableEnabled_result(this);
     }
 
     @Override
     public void clear() {
+      setSuccessIsSet(false);
+      this.success = false;
       this.io = null;
     }
 
+    public boolean isSuccess() {
+      return this.success;
+    }
+
+    public isTableEnabled_result setSuccess(boolean success) {
+      this.success = success;
+      setSuccessIsSet(true);
+      return this;
+    }
+
+    public void unsetSuccess() {
+      __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __SUCCESS_ISSET_ID);
+    }
+
+    /** Returns true if field success is set (has been assigned a value) and false otherwise */
+    public boolean isSetSuccess() {
+      return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __SUCCESS_ISSET_ID);
+    }
+
+    public void setSuccessIsSet(boolean value) {
+      __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __SUCCESS_ISSET_ID, value);
+    }
+
     @org.apache.thrift.annotation.Nullable
     public TIOError getIo() {
       return this.io;
     }
 
-    public disableTable_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) {
+    public isTableEnabled_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) {
       this.io = io;
       return this;
     }
@@ -39688,6 +40619,14 @@ public class THBaseService {
 
     public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
       switch (field) {
+      case SUCCESS:
+        if (value == null) {
+          unsetSuccess();
+        } else {
+          setSuccess((java.lang.Boolean)value);
+        }
+        break;
+
       case IO:
         if (value == null) {
           unsetIo();
@@ -39702,6 +40641,9 @@ public class THBaseService {
     @org.apache.thrift.annotation.Nullable
     public java.lang.Object getFieldValue(_Fields field) {
       switch (field) {
+      case SUCCESS:
+        return isSuccess();
+
       case IO:
         return getIo();
 
@@ -39716,6 +40658,8 @@ public class THBaseService {
       }
 
       switch (field) {
+      case SUCCESS:
+        return isSetSuccess();
       case IO:
         return isSetIo();
       }
@@ -39726,17 +40670,26 @@ public class THBaseService {
     public boolean equals(java.lang.Object that) {
       if (that == null)
         return false;
-      if (that instanceof disableTable_result)
-        return this.equals((disableTable_result)that);
+      if (that instanceof isTableEnabled_result)
+        return this.equals((isTableEnabled_result)that);
       return false;
     }
 
-    public boolean equals(disableTable_result that) {
+    public boolean equals(isTableEnabled_result that) {
       if (that == null)
         return false;
       if (this == that)
         return true;
 
+      boolean this_present_success = true;
+      boolean that_present_success = true;
+      if (this_present_success || that_present_success) {
+        if (!(this_present_success && that_present_success))
+          return false;
+        if (this.success != that.success)
+          return false;
+      }
+
       boolean this_present_io = true && this.isSetIo();
       boolean that_present_io = true && that.isSetIo();
       if (this_present_io || that_present_io) {
@@ -39753,6 +40706,8 @@ public class THBaseService {
     public int hashCode() {
       int hashCode = 1;
 
+      hashCode = hashCode * 8191 + ((success) ? 131071 : 524287);
+
       hashCode = hashCode * 8191 + ((isSetIo()) ? 131071 : 524287);
       if (isSetIo())
         hashCode = hashCode * 8191 + io.hashCode();
@@ -39761,13 +40716,23 @@ public class THBaseService {
     }
 
     @Override
-    public int compareTo(disableTable_result other) {
+    public int compareTo(isTableEnabled_result other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
 
       int lastComparison = 0;
 
+      lastComparison = java.lang.Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetSuccess()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
       lastComparison = java.lang.Boolean.valueOf(isSetIo()).compareTo(other.isSetIo());
       if (lastComparison != 0) {
         return lastComparison;
@@ -39796,9 +40761,13 @@ public class THBaseService {
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("disableTable_result(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("isTableEnabled_result(");
       boolean first = true;
 
+      sb.append("success:");
+      sb.append(this.success);
+      first = false;
+      if (!first) sb.append(", ");
       sb.append("io:");
       if (this.io == null) {
         sb.append("null");
@@ -39825,21 +40794,23 @@ public class THBaseService {
 
     private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
       try {
+        // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+        __isset_bitfield = 0;
         read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
       } catch (org.apache.thrift.TException te) {
         throw new java.io.IOException(te);
       }
     }
 
-    private static class disableTable_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public disableTable_resultStandardScheme getScheme() {
-        return new disableTable_resultStandardScheme();
+    private static class isTableEnabled_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public isTableEnabled_resultStandardScheme getScheme() {
+        return new isTableEnabled_resultStandardScheme();
       }
     }
 
-    private static class disableTable_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<disableTable_result> {
+    private static class isTableEnabled_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<isTableEnabled_result> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, disableTable_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, isTableEnabled_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -39849,6 +40820,14 @@ public class THBaseService {
             break;
           }
           switch (schemeField.id) {
+            case 0: // SUCCESS
+              if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+                struct.success = iprot.readBool();
+                struct.setSuccessIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
             case 1: // IO
               if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
                 struct.io = new TIOError();
@@ -39869,10 +40848,15 @@ public class THBaseService {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, disableTable_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, isTableEnabled_result struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.isSetSuccess()) {
+          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+          oprot.writeBool(struct.success);
+          oprot.writeFieldEnd();
+        }
         if (struct.io != null) {
           oprot.writeFieldBegin(IO_FIELD_DESC);
           struct.io.write(oprot);
@@ -39884,32 +40868,42 @@ public class THBaseService {
 
     }
 
-    private static class disableTable_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public disableTable_resultTupleScheme getScheme() {
-        return new disableTable_resultTupleScheme();
+    private static class isTableEnabled_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public isTableEnabled_resultTupleScheme getScheme() {
+        return new isTableEnabled_resultTupleScheme();
       }
     }
 
-    private static class disableTable_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<disableTable_result> {
+    private static class isTableEnabled_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<isTableEnabled_result> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, disableTable_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, isTableEnabled_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet optionals = new java.util.BitSet();
-        if (struct.isSetIo()) {
+        if (struct.isSetSuccess()) {
           optionals.set(0);
         }
-        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetIo()) {
+          optionals.set(1);
+        }
+        oprot.writeBitSet(optionals, 2);
+        if (struct.isSetSuccess()) {
+          oprot.writeBool(struct.success);
+        }
         if (struct.isSetIo()) {
           struct.io.write(oprot);
         }
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, disableTable_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, isTableEnabled_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
-        java.util.BitSet incoming = iprot.readBitSet(1);
+        java.util.BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
+          struct.success = iprot.readBool();
+          struct.setSuccessIsSet(true);
+        }
+        if (incoming.get(1)) {
           struct.io = new TIOError();
           struct.io.read(iprot);
           struct.setIoIsSet(true);
@@ -39922,13 +40916,13 @@ public class THBaseService {
     }
   }
 
-  public static class isTableEnabled_args implements org.apache.thrift.TBase<isTableEnabled_args, isTableEnabled_args._Fields>, java.io.Serializable, Cloneable, Comparable<isTableEnabled_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("isTableEnabled_args");
+  public static class isTableDisabled_args implements org.apache.thrift.TBase<isTableDisabled_args, isTableDisabled_args._Fields>, java.io.Serializable, Cloneable, Comparable<isTableDisabled_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("isTableDisabled_args");
 
     private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new isTableEnabled_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new isTableEnabled_argsTupleSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new isTableDisabled_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new isTableDisabled_argsTupleSchemeFactory();
 
     /**
      * the tablename to check
@@ -40005,13 +40999,13 @@ public class THBaseService {
       tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TTableName.class)));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(isTableEnabled_args.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(isTableDisabled_args.class, metaDataMap);
     }
 
-    public isTableEnabled_args() {
+    public isTableDisabled_args() {
     }
 
-    public isTableEnabled_args(
+    public isTableDisabled_args(
       TTableName tableName)
     {
       this();
@@ -40021,14 +41015,14 @@ public class THBaseService {
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public isTableEnabled_args(isTableEnabled_args other) {
+    public isTableDisabled_args(isTableDisabled_args other) {
       if (other.isSetTableName()) {
         this.tableName = new TTableName(other.tableName);
       }
     }
 
-    public isTableEnabled_args deepCopy() {
-      return new isTableEnabled_args(this);
+    public isTableDisabled_args deepCopy() {
+      return new isTableDisabled_args(this);
     }
 
     @Override
@@ -40047,7 +41041,7 @@ public class THBaseService {
     /**
      * the tablename to check
      */
-    public isTableEnabled_args setTableName(@org.apache.thrift.annotation.Nullable TTableName tableName) {
+    public isTableDisabled_args setTableName(@org.apache.thrift.annotation.Nullable TTableName tableName) {
       this.tableName = tableName;
       return this;
     }
@@ -40107,12 +41101,12 @@ public class THBaseService {
     public boolean equals(java.lang.Object that) {
       if (that == null)
         return false;
-      if (that instanceof isTableEnabled_args)
-        return this.equals((isTableEnabled_args)that);
+      if (that instanceof isTableDisabled_args)
+        return this.equals((isTableDisabled_args)that);
       return false;
     }
 
-    public boolean equals(isTableEnabled_args that) {
+    public boolean equals(isTableDisabled_args that) {
       if (that == null)
         return false;
       if (this == that)
@@ -40142,7 +41136,7 @@ public class THBaseService {
     }
 
     @Override
-    public int compareTo(isTableEnabled_args other) {
+    public int compareTo(isTableDisabled_args other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -40177,7 +41171,7 @@ public class THBaseService {
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("isTableEnabled_args(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("isTableDisabled_args(");
       boolean first = true;
 
       sb.append("tableName:");
@@ -40218,15 +41212,15 @@ public class THBaseService {
       }
     }
 
-    private static class isTableEnabled_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public isTableEnabled_argsStandardScheme getScheme() {
-        return new isTableEnabled_argsStandardScheme();
+    private static class isTableDisabled_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public isTableDisabled_argsStandardScheme getScheme() {
+        return new isTableDisabled_argsStandardScheme();
       }
     }
 
-    private static class isTableEnabled_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<isTableEnabled_args> {
+    private static class isTableDisabled_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<isTableDisabled_args> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, isTableEnabled_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, isTableDisabled_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -40256,7 +41250,7 @@ public class THBaseService {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, isTableEnabled_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, isTableDisabled_args struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -40271,22 +41265,22 @@ public class THBaseService {
 
     }
 
-    private static class isTableEnabled_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public isTableEnabled_argsTupleScheme getScheme() {
-        return new isTableEnabled_argsTupleScheme();
+    private static class isTableDisabled_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public isTableDisabled_argsTupleScheme getScheme() {
+        return new isTableDisabled_argsTupleScheme();
       }
     }
 
-    private static class isTableEnabled_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<isTableEnabled_args> {
+    private static class isTableDisabled_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<isTableDisabled_args> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, isTableEnabled_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, isTableDisabled_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         struct.tableName.write(oprot);
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, isTableEnabled_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, isTableDisabled_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         struct.tableName = new TTableName();
         struct.tableName.read(iprot);
@@ -40299,14 +41293,14 @@ public class THBaseService {
     }
   }
 
-  public static class isTableEnabled_result implements org.apache.thrift.TBase<isTableEnabled_result, isTableEnabled_result._Fields>, java.io.Serializable, Cloneable, Comparable<isTableEnabled_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("isTableEnabled_result");
+  public static class isTableDisabled_result implements org.apache.thrift.TBase<isTableDisabled_result, isTableDisabled_result._Fields>, java.io.Serializable, Cloneable, Comparable<isTableDisabled_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("isTableDisabled_result");
 
     private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.BOOL, (short)0);
     private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new isTableEnabled_resultStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new isTableEnabled_resultTupleSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new isTableDisabled_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new isTableDisabled_resultTupleSchemeFactory();
 
     public boolean success; // required
     public @org.apache.thrift.annotation.Nullable TIOError io; // required
@@ -40385,13 +41379,13 @@ public class THBaseService {
       tmpMap.put(_Fields.IO, new org.apache.thrift.meta_data.FieldMetaData("io", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TIOError.class)));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(isTableEnabled_result.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(isTableDisabled_result.class, metaDataMap);
     }
 
-    public isTableEnabled_result() {
+    public isTableDisabled_result() {
     }
 
-    public isTableEnabled_result(
+    public isTableDisabled_result(
       boolean success,
       TIOError io)
     {
@@ -40404,7 +41398,7 @@ public class THBaseService {
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public isTableEnabled_result(isTableEnabled_result other) {
+    public isTableDisabled_result(isTableDisabled_result other) {
       __isset_bitfield = other.__isset_bitfield;
       this.success = other.success;
       if (other.isSetIo()) {
@@ -40412,8 +41406,8 @@ public class THBaseService {
       }
     }
 
-    public isTableEnabled_result deepCopy() {
-      return new isTableEnabled_result(this);
+    public isTableDisabled_result deepCopy() {
+      return new isTableDisabled_result(this);
     }
 
     @Override
@@ -40427,7 +41421,7 @@ public class THBaseService {
       return this.success;
     }
 
-    public isTableEnabled_result setSuccess(boolean success) {
+    public isTableDisabled_result setSuccess(boolean success) {
       this.success = success;
       setSuccessIsSet(true);
       return this;
@@ -40451,7 +41445,7 @@ public class THBaseService {
       return this.io;
     }
 
-    public isTableEnabled_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) {
+    public isTableDisabled_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) {
       this.io = io;
       return this;
     }
@@ -40524,12 +41518,12 @@ public class THBaseService {
     public boolean equals(java.lang.Object that) {
       if (that == null)
         return false;
-      if (that instanceof isTableEnabled_result)
-        return this.equals((isTableEnabled_result)that);
+      if (that instanceof isTableDisabled_result)
+        return this.equals((isTableDisabled_result)that);
       return false;
     }
 
-    public boolean equals(isTableEnabled_result that) {
+    public boolean equals(isTableDisabled_result that) {
       if (that == null)
         return false;
       if (this == that)
@@ -40570,7 +41564,7 @@ public class THBaseService {
     }
 
     @Override
-    public int compareTo(isTableEnabled_result other) {
+    public int compareTo(isTableDisabled_result other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -40615,7 +41609,7 @@ public class THBaseService {
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("isTableEnabled_result(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("isTableDisabled_result(");
       boolean first = true;
 
       sb.append("success:");
@@ -40656,15 +41650,15 @@ public class THBaseService {
       }
     }
 
-    private static class isTableEnabled_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public isTableEnabled_resultStandardScheme getScheme() {
-        return new isTableEnabled_resultStandardScheme();
+    private static class isTableDisabled_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public isTableDisabled_resultStandardScheme getScheme() {
+        return new isTableDisabled_resultStandardScheme();
       }
     }
 
-    private static class isTableEnabled_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<isTableEnabled_result> {
+    private static class isTableDisabled_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<isTableDisabled_result> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, isTableEnabled_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, isTableDisabled_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -40702,7 +41696,7 @@ public class THBaseService {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, isTableEnabled_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, isTableDisabled_result struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -40722,16 +41716,16 @@ public class THBaseService {
 
     }
 
-    private static class isTableEnabled_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public isTableEnabled_resultTupleScheme getScheme() {
-        return new isTableEnabled_resultTupleScheme();
+    private static class isTableDisabled_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public isTableDisabled_resultTupleScheme getScheme() {
+        return new isTableDisabled_resultTupleScheme();
       }
     }
 
-    private static class isTableEnabled_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<isTableEnabled_result> {
+    private static class isTableDisabled_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<isTableDisabled_result> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, isTableEnabled_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, isTableDisabled_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet optionals = new java.util.BitSet();
         if (struct.isSetSuccess()) {
@@ -40750,7 +41744,7 @@ public class THBaseService {
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, isTableEnabled_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, isTableDisabled_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
@@ -40770,13 +41764,13 @@ public class THBaseService {
     }
   }
 
-  public static class isTableDisabled_args implements org.apache.thrift.TBase<isTableDisabled_args, isTableDisabled_args._Fields>, java.io.Serializable, Cloneable, Comparable<isTableDisabled_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("isTableDisabled_args");
+  public static class isTableAvailable_args implements org.apache.thrift.TBase<isTableAvailable_args, isTableAvailable_args._Fields>, java.io.Serializable, Cloneable, Comparable<isTableAvailable_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("isTableAvailable_args");
 
     private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new isTableDisabled_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new isTableDisabled_argsTupleSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new isTableAvailable_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new isTableAvailable_argsTupleSchemeFactory();
 
     /**
      * the tablename to check
@@ -40853,13 +41847,13 @@ public class THBaseService {
       tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TTableName.class)));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(isTableDisabled_args.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(isTableAvailable_args.class, metaDataMap);
     }
 
-    public isTableDisabled_args() {
+    public isTableAvailable_args() {
     }
 
-    public isTableDisabled_args(
+    public isTableAvailable_args(
       TTableName tableName)
     {
       this();
@@ -40869,14 +41863,14 @@ public class THBaseService {
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public isTableDisabled_args(isTableDisabled_args other) {
+    public isTableAvailable_args(isTableAvailable_args other) {
       if (other.isSetTableName()) {
         this.tableName = new TTableName(other.tableName);
       }
     }
 
-    public isTableDisabled_args deepCopy() {
-      return new isTableDisabled_args(this);
+    public isTableAvailable_args deepCopy() {
+      return new isTableAvailable_args(this);
     }
 
     @Override
@@ -40895,7 +41889,7 @@ public class THBaseService {
     /**
      * the tablename to check
      */
-    public isTableDisabled_args setTableName(@org.apache.thrift.annotation.Nullable TTableName tableName) {
+    public isTableAvailable_args setTableName(@org.apache.thrift.annotation.Nullable TTableName tableName) {
       this.tableName = tableName;
       return this;
     }
@@ -40955,12 +41949,12 @@ public class THBaseService {
     public boolean equals(java.lang.Object that) {
       if (that == null)
         return false;
-      if (that instanceof isTableDisabled_args)
-        return this.equals((isTableDisabled_args)that);
+      if (that instanceof isTableAvailable_args)
+        return this.equals((isTableAvailable_args)that);
       return false;
     }
 
-    public boolean equals(isTableDisabled_args that) {
+    public boolean equals(isTableAvailable_args that) {
       if (that == null)
         return false;
       if (this == that)
@@ -40990,7 +41984,7 @@ public class THBaseService {
     }
 
     @Override
-    public int compareTo(isTableDisabled_args other) {
+    public int compareTo(isTableAvailable_args other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -41025,7 +42019,7 @@ public class THBaseService {
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("isTableDisabled_args(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("isTableAvailable_args(");
       boolean first = true;
 
       sb.append("tableName:");
@@ -41066,15 +42060,15 @@ public class THBaseService {
       }
     }
 
-    private static class isTableDisabled_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public isTableDisabled_argsStandardScheme getScheme() {
-        return new isTableDisabled_argsStandardScheme();
+    private static class isTableAvailable_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public isTableAvailable_argsStandardScheme getScheme() {
+        return new isTableAvailable_argsStandardScheme();
       }
     }
 
-    private static class isTableDisabled_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<isTableDisabled_args> {
+    private static class isTableAvailable_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<isTableAvailable_args> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, isTableDisabled_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, isTableAvailable_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -41104,7 +42098,7 @@ public class THBaseService {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, isTableDisabled_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, isTableAvailable_args struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -41119,22 +42113,22 @@ public class THBaseService {
 
     }
 
-    private static class isTableDisabled_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public isTableDisabled_argsTupleScheme getScheme() {
-        return new isTableDisabled_argsTupleScheme();
+    private static class isTableAvailable_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public isTableAvailable_argsTupleScheme getScheme() {
+        return new isTableAvailable_argsTupleScheme();
       }
     }
 
-    private static class isTableDisabled_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<isTableDisabled_args> {
+    private static class isTableAvailable_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<isTableAvailable_args> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, isTableDisabled_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, isTableAvailable_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         struct.tableName.write(oprot);
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, isTableDisabled_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, isTableAvailable_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         struct.tableName = new TTableName();
         struct.tableName.read(iprot);
@@ -41147,14 +42141,14 @@ public class THBaseService {
     }
   }
 
-  public static class isTableDisabled_result implements org.apache.thrift.TBase<isTableDisabled_result, isTableDisabled_result._Fields>, java.io.Serializable, Cloneable, Comparable<isTableDisabled_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("isTableDisabled_result");
+  public static class isTableAvailable_result implements org.apache.thrift.TBase<isTableAvailable_result, isTableAvailable_result._Fields>, java.io.Serializable, Cloneable, Comparable<isTableAvailable_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("isTableAvailable_result");
 
     private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.BOOL, (short)0);
     private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new isTableDisabled_resultStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new isTableDisabled_resultTupleSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new isTableAvailable_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new isTableAvailable_resultTupleSchemeFactory();
 
     public boolean success; // required
     public @org.apache.thrift.annotation.Nullable TIOError io; // required
@@ -41233,13 +42227,13 @@ public class THBaseService {
       tmpMap.put(_Fields.IO, new org.apache.thrift.meta_data.FieldMetaData("io", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TIOError.class)));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(isTableDisabled_result.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(isTableAvailable_result.class, metaDataMap);
     }
 
-    public isTableDisabled_result() {
+    public isTableAvailable_result() {
     }
 
-    public isTableDisabled_result(
+    public isTableAvailable_result(
       boolean success,
       TIOError io)
     {
@@ -41252,7 +42246,7 @@ public class THBaseService {
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public isTableDisabled_result(isTableDisabled_result other) {
+    public isTableAvailable_result(isTableAvailable_result other) {
       __isset_bitfield = other.__isset_bitfield;
       this.success = other.success;
       if (other.isSetIo()) {
@@ -41260,8 +42254,8 @@ public class THBaseService {
       }
     }
 
-    public isTableDisabled_result deepCopy() {
-      return new isTableDisabled_result(this);
+    public isTableAvailable_result deepCopy() {
+      return new isTableAvailable_result(this);
     }
 
     @Override
@@ -41275,7 +42269,7 @@ public class THBaseService {
       return this.success;
     }
 
-    public isTableDisabled_result setSuccess(boolean success) {
+    public isTableAvailable_result setSuccess(boolean success) {
       this.success = success;
       setSuccessIsSet(true);
       return this;
@@ -41299,7 +42293,7 @@ public class THBaseService {
       return this.io;
     }
 
-    public isTableDisabled_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) {
+    public isTableAvailable_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) {
       this.io = io;
       return this;
     }
@@ -41372,12 +42366,12 @@ public class THBaseService {
     public boolean equals(java.lang.Object that) {
       if (that == null)
         return false;
-      if (that instanceof isTableDisabled_result)
-        return this.equals((isTableDisabled_result)that);
+      if (that instanceof isTableAvailable_result)
+        return this.equals((isTableAvailable_result)that);
       return false;
     }
 
-    public boolean equals(isTableDisabled_result that) {
+    public boolean equals(isTableAvailable_result that) {
       if (that == null)
         return false;
       if (this == that)
@@ -41418,7 +42412,7 @@ public class THBaseService {
     }
 
     @Override
-    public int compareTo(isTableDisabled_result other) {
+    public int compareTo(isTableAvailable_result other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -41463,7 +42457,7 @@ public class THBaseService {
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("isTableDisabled_result(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("isTableAvailable_result(");
       boolean first = true;
 
       sb.append("success:");
@@ -41504,15 +42498,15 @@ public class THBaseService {
       }
     }
 
-    private static class isTableDisabled_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public isTableDisabled_resultStandardScheme getScheme() {
-        return new isTableDisabled_resultStandardScheme();
+    private static class isTableAvailable_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public isTableAvailable_resultStandardScheme getScheme() {
+        return new isTableAvailable_resultStandardScheme();
       }
     }
 
-    private static class isTableDisabled_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<isTableDisabled_result> {
+    private static class isTableAvailable_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<isTableAvailable_result> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, isTableDisabled_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, isTableAvailable_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -41550,7 +42544,7 @@ public class THBaseService {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, isTableDisabled_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, isTableAvailable_result struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -41570,16 +42564,16 @@ public class THBaseService {
 
     }
 
-    private static class isTableDisabled_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public isTableDisabled_resultTupleScheme getScheme() {
-        return new isTableDisabled_resultTupleScheme();
+    private static class isTableAvailable_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public isTableAvailable_resultTupleScheme getScheme() {
+        return new isTableAvailable_resultTupleScheme();
       }
     }
 
-    private static class isTableDisabled_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<isTableDisabled_result> {
+    private static class isTableAvailable_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<isTableAvailable_result> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, isTableDisabled_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, isTableAvailable_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet optionals = new java.util.BitSet();
         if (struct.isSetSuccess()) {
@@ -41598,7 +42592,7 @@ public class THBaseService {
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, isTableDisabled_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, isTableAvailable_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
@@ -41618,25 +42612,34 @@ public class THBaseService {
     }
   }
 
-  public static class isTableAvailable_args implements org.apache.thrift.TBase<isTableAvailable_args, isTableAvailable_args._Fields>, java.io.Serializable, Cloneable, Comparable<isTableAvailable_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("isTableAvailable_args");
+  public static class isTableAvailableWithSplit_args implements org.apache.thrift.TBase<isTableAvailableWithSplit_args, isTableAvailableWithSplit_args._Fields>, java.io.Serializable, Cloneable, Comparable<isTableAvailableWithSplit_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("isTableAvailableWithSplit_args");
 
     private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+    private static final org.apache.thrift.protocol.TField SPLIT_KEYS_FIELD_DESC = new org.apache.thrift.protocol.TField("splitKeys", org.apache.thrift.protocol.TType.LIST, (short)2);
 
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new isTableAvailable_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new isTableAvailable_argsTupleSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new isTableAvailableWithSplit_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new isTableAvailableWithSplit_argsTupleSchemeFactory();
 
     /**
      * the tablename to check
      */
     public @org.apache.thrift.annotation.Nullable TTableName tableName; // required
+    /**
+     * keys to check if the table has been created with all split keys
+     */
+    public @org.apache.thrift.annotation.Nullable java.util.List<java.nio.ByteBuffer> splitKeys; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
       /**
        * the tablename to check
        */
-      TABLE_NAME((short)1, "tableName");
+      TABLE_NAME((short)1, "tableName"),
+      /**
+       * keys to check if the table has been created with all split keys
+       */
+      SPLIT_KEYS((short)2, "splitKeys");
 
       private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
 
@@ -41654,6 +42657,8 @@ public class THBaseService {
         switch(fieldId) {
           case 1: // TABLE_NAME
             return TABLE_NAME;
+          case 2: // SPLIT_KEYS
+            return SPLIT_KEYS;
           default:
             return null;
         }
@@ -41700,36 +42705,46 @@ public class THBaseService {
       java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
       tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TTableName.class)));
+      tmpMap.put(_Fields.SPLIT_KEYS, new org.apache.thrift.meta_data.FieldMetaData("splitKeys", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+              new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING              , true))));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(isTableAvailable_args.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(isTableAvailableWithSplit_args.class, metaDataMap);
     }
 
-    public isTableAvailable_args() {
+    public isTableAvailableWithSplit_args() {
     }
 
-    public isTableAvailable_args(
-      TTableName tableName)
+    public isTableAvailableWithSplit_args(
+      TTableName tableName,
+      java.util.List<java.nio.ByteBuffer> splitKeys)
     {
       this();
       this.tableName = tableName;
+      this.splitKeys = splitKeys;
     }
 
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public isTableAvailable_args(isTableAvailable_args other) {
+    public isTableAvailableWithSplit_args(isTableAvailableWithSplit_args other) {
       if (other.isSetTableName()) {
         this.tableName = new TTableName(other.tableName);
       }
+      if (other.isSetSplitKeys()) {
+        java.util.List<java.nio.ByteBuffer> __this__splitKeys = new java.util.ArrayList<java.nio.ByteBuffer>(other.splitKeys);
+        this.splitKeys = __this__splitKeys;
+      }
     }
 
-    public isTableAvailable_args deepCopy() {
-      return new isTableAvailable_args(this);
+    public isTableAvailableWithSplit_args deepCopy() {
+      return new isTableAvailableWithSplit_args(this);
     }
 
     @Override
     public void clear() {
       this.tableName = null;
+      this.splitKeys = null;
     }
 
     /**
@@ -41743,7 +42758,7 @@ public class THBaseService {
     /**
      * the tablename to check
      */
-    public isTableAvailable_args setTableName(@org.apache.thrift.annotation.Nullable TTableName tableName) {
+    public isTableAvailableWithSplit_args setTableName(@org.apache.thrift.annotation.Nullable TTableName tableName) {
       this.tableName = tableName;
       return this;
     }
@@ -41763,6 +42778,53 @@ public class THBaseService {
       }
     }
 
+    public int getSplitKeysSize() {
+      return (this.splitKeys == null) ? 0 : this.splitKeys.size();
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.util.Iterator<java.nio.ByteBuffer> getSplitKeysIterator() {
+      return (this.splitKeys == null) ? null : this.splitKeys.iterator();
+    }
+
+    public void addToSplitKeys(java.nio.ByteBuffer elem) {
+      if (this.splitKeys == null) {
+        this.splitKeys = new java.util.ArrayList<java.nio.ByteBuffer>();
+      }
+      this.splitKeys.add(elem);
+    }
+
+    /**
+     * keys to check if the table has been created with all split keys
+     */
+    @org.apache.thrift.annotation.Nullable
+    public java.util.List<java.nio.ByteBuffer> getSplitKeys() {
+      return this.splitKeys;
+    }
+
+    /**
+     * keys to check if the table has been created with all split keys
+     */
+    public isTableAvailableWithSplit_args setSplitKeys(@org.apache.thrift.annotation.Nullable java.util.List<java.nio.ByteBuffer> splitKeys) {
+      this.splitKeys = splitKeys;
+      return this;
+    }
+
+    public void unsetSplitKeys() {
+      this.splitKeys = null;
+    }
+
+    /** Returns true if field splitKeys is set (has been assigned a value) and false otherwise */
+    public boolean isSetSplitKeys() {
+      return this.splitKeys != null;
+    }
+
+    public void setSplitKeysIsSet(boolean value) {
+      if (!value) {
+        this.splitKeys = null;
+      }
+    }
+
     public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
       switch (field) {
       case TABLE_NAME:
@@ -41773,6 +42835,14 @@ public class THBaseService {
         }
         break;
 
+      case SPLIT_KEYS:
+        if (value == null) {
+          unsetSplitKeys();
+        } else {
+          setSplitKeys((java.util.List<java.nio.ByteBuffer>)value);
+        }
+        break;
+
       }
     }
 
@@ -41782,6 +42852,9 @@ public class THBaseService {
       case TABLE_NAME:
         return getTableName();
 
+      case SPLIT_KEYS:
+        return getSplitKeys();
+
       }
       throw new java.lang.IllegalStateException();
     }
@@ -41795,6 +42868,8 @@ public class THBaseService {
       switch (field) {
       case TABLE_NAME:
         return isSetTableName();
+      case SPLIT_KEYS:
+        return isSetSplitKeys();
       }
       throw new java.lang.IllegalStateException();
     }
@@ -41803,12 +42878,12 @@ public class THBaseService {
     public boolean equals(java.lang.Object that) {
       if (that == null)
         return false;
-      if (that instanceof isTableAvailable_args)
-        return this.equals((isTableAvailable_args)that);
+      if (that instanceof isTableAvailableWithSplit_args)
+        return this.equals((isTableAvailableWithSplit_args)that);
       return false;
     }
 
-    public boolean equals(isTableAvailable_args that) {
+    public boolean equals(isTableAvailableWithSplit_args that) {
       if (that == null)
         return false;
       if (this == that)
@@ -41823,6 +42898,15 @@ public class THBaseService {
           return false;
       }
 
+      boolean this_present_splitKeys = true && this.isSetSplitKeys();
+      boolean that_present_splitKeys = true && that.isSetSplitKeys();
+      if (this_present_splitKeys || that_present_splitKeys) {
+        if (!(this_present_splitKeys && that_present_splitKeys))
+          return false;
+        if (!this.splitKeys.equals(that.splitKeys))
+          return false;
+      }
+
       return true;
     }
 
@@ -41834,11 +42918,15 @@ public class THBaseService {
       if (isSetTableName())
         hashCode = hashCode * 8191 + tableName.hashCode();
 
+      hashCode = hashCode * 8191 + ((isSetSplitKeys()) ? 131071 : 524287);
+      if (isSetSplitKeys())
+        hashCode = hashCode * 8191 + splitKeys.hashCode();
+
       return hashCode;
     }
 
     @Override
-    public int compareTo(isTableAvailable_args other) {
+    public int compareTo(isTableAvailableWithSplit_args other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -41855,6 +42943,16 @@ public class THBaseService {
           return lastComparison;
         }
       }
+      lastComparison = java.lang.Boolean.valueOf(isSetSplitKeys()).compareTo(other.isSetSplitKeys());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetSplitKeys()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.splitKeys, other.splitKeys);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
       return 0;
     }
 
@@ -41873,7 +42971,7 @@ public class THBaseService {
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("isTableAvailable_args(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("isTableAvailableWithSplit_args(");
       boolean first = true;
 
       sb.append("tableName:");
@@ -41883,6 +42981,14 @@ public class THBaseService {
         sb.append(this.tableName);
       }
       first = false;
+      if (!first) sb.append(", ");
+      sb.append("splitKeys:");
+      if (this.splitKeys == null) {
+        sb.append("null");
+      } else {
+        org.apache.thrift.TBaseHelper.toString(this.splitKeys, sb);
+      }
+      first = false;
       sb.append(")");
       return sb.toString();
     }
@@ -41914,15 +43020,15 @@ public class THBaseService {
       }
     }
 
-    private static class isTableAvailable_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public isTableAvailable_argsStandardScheme getScheme() {
-        return new isTableAvailable_argsStandardScheme();
+    private static class isTableAvailableWithSplit_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public isTableAvailableWithSplit_argsStandardScheme getScheme() {
+        return new isTableAvailableWithSplit_argsStandardScheme();
       }
     }
 
-    private static class isTableAvailable_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<isTableAvailable_args> {
+    private static class isTableAvailableWithSplit_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<isTableAvailableWithSplit_args> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, isTableAvailable_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, isTableAvailableWithSplit_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -41941,6 +43047,24 @@ public class THBaseService {
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
               break;
+            case 2: // SPLIT_KEYS
+              if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+                {
+                  org.apache.thrift.protocol.TList _list326 = iprot.readListBegin();
+                  struct.splitKeys = new java.util.ArrayList<java.nio.ByteBuffer>(_list326.size);
+                  @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem327;
+                  for (int _i328 = 0; _i328 < _list326.size; ++_i328)
+                  {
+                    _elem327 = iprot.readBinary();
+                    struct.splitKeys.add(_elem327);
+                  }
+                  iprot.readListEnd();
+                }
+                struct.setSplitKeysIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
             default:
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
           }
@@ -41952,7 +43076,7 @@ public class THBaseService {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, isTableAvailable_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, isTableAvailableWithSplit_args struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -41961,32 +43085,72 @@ public class THBaseService {
           struct.tableName.write(oprot);
           oprot.writeFieldEnd();
         }
+        if (struct.splitKeys != null) {
+          oprot.writeFieldBegin(SPLIT_KEYS_FIELD_DESC);
+          {
+            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.splitKeys.size()));
+            for (java.nio.ByteBuffer _iter329 : struct.splitKeys)
+            {
+              oprot.writeBinary(_iter329);
+            }
+            oprot.writeListEnd();
+          }
+          oprot.writeFieldEnd();
+        }
         oprot.writeFieldStop();
         oprot.writeStructEnd();
       }
 
     }
 
-    private static class isTableAvailable_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public isTableAvailable_argsTupleScheme getScheme() {
-        return new isTableAvailable_argsTupleScheme();
+    private static class isTableAvailableWithSplit_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public isTableAvailableWithSplit_argsTupleScheme getScheme() {
+        return new isTableAvailableWithSplit_argsTupleScheme();
       }
     }
 
-    private static class isTableAvailable_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<isTableAvailable_args> {
+    private static class isTableAvailableWithSplit_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<isTableAvailableWithSplit_args> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, isTableAvailable_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, isTableAvailableWithSplit_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         struct.tableName.write(oprot);
+        java.util.BitSet optionals = new java.util.BitSet();
+        if (struct.isSetSplitKeys()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetSplitKeys()) {
+          {
+            oprot.writeI32(struct.splitKeys.size());
+            for (java.nio.ByteBuffer _iter330 : struct.splitKeys)
+            {
+              oprot.writeBinary(_iter330);
+            }
+          }
+        }
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, isTableAvailable_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, isTableAvailableWithSplit_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         struct.tableName = new TTableName();
         struct.tableName.read(iprot);
         struct.setTableNameIsSet(true);
+        java.util.BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          {
+            org.apache.thrift.protocol.TList _list331 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.splitKeys = new java.util.ArrayList<java.nio.ByteBuffer>(_list331.size);
+            @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem332;
+            for (int _i333 = 0; _i333 < _list331.size; ++_i333)
+            {
+              _elem332 = iprot.readBinary();
+              struct.splitKeys.add(_elem332);
+            }
+          }
+          struct.setSplitKeysIsSet(true);
+        }
       }
     }
 
@@ -41995,14 +43159,14 @@ public class THBaseService {
     }
   }
 
-  public static class isTableAvailable_result implements org.apache.thrift.TBase<isTableAvailable_result, isTableAvailable_result._Fields>, java.io.Serializable, Cloneable, Comparable<isTableAvailable_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("isTableAvailable_result");
+  public static class isTableAvailableWithSplit_result implements org.apache.thrift.TBase<isTableAvailableWithSplit_result, isTableAvailableWithSplit_result._Fields>, java.io.Serializable, Cloneable, Comparable<isTableAvailableWithSplit_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("isTableAvailableWithSplit_result");
 
     private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.BOOL, (short)0);
     private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new isTableAvailable_resultStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new isTableAvailable_resultTupleSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new isTableAvailableWithSplit_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new isTableAvailableWithSplit_resultTupleSchemeFactory();
 
     public boolean success; // required
     public @org.apache.thrift.annotation.Nullable TIOError io; // required
@@ -42081,13 +43245,13 @@ public class THBaseService {
       tmpMap.put(_Fields.IO, new org.apache.thrift.meta_data.FieldMetaData("io", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TIOError.class)));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(isTableAvailable_result.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(isTableAvailableWithSplit_result.class, metaDataMap);
     }
 
-    public isTableAvailable_result() {
+    public isTableAvailableWithSplit_result() {
     }
 
-    public isTableAvailable_result(
+    public isTableAvailableWithSplit_result(
       boolean success,
       TIOError io)
     {
@@ -42100,7 +43264,7 @@ public class THBaseService {
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public isTableAvailable_result(isTableAvailable_result other) {
+    public isTableAvailableWithSplit_result(isTableAvailableWithSplit_result other) {
       __isset_bitfield = other.__isset_bitfield;
       this.success = other.success;
       if (other.isSetIo()) {
@@ -42108,8 +43272,8 @@ public class THBaseService {
       }
     }
 
-    public isTableAvailable_result deepCopy() {
-      return new isTableAvailable_result(this);
+    public isTableAvailableWithSplit_result deepCopy() {
+      return new isTableAvailableWithSplit_result(this);
     }
 
     @Override
@@ -42123,7 +43287,7 @@ public class THBaseService {
       return this.success;
     }
 
-    public isTableAvailable_result setSuccess(boolean success) {
+    public isTableAvailableWithSplit_result setSuccess(boolean success) {
       this.success = success;
       setSuccessIsSet(true);
       return this;
@@ -42147,7 +43311,7 @@ public class THBaseService {
       return this.io;
     }
 
-    public isTableAvailable_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) {
+    public isTableAvailableWithSplit_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) {
       this.io = io;
       return this;
     }
@@ -42220,12 +43384,12 @@ public class THBaseService {
     public boolean equals(java.lang.Object that) {
       if (that == null)
         return false;
-      if (that instanceof isTableAvailable_result)
-        return this.equals((isTableAvailable_result)that);
+      if (that instanceof isTableAvailableWithSplit_result)
+        return this.equals((isTableAvailableWithSplit_result)that);
       return false;
     }
 
-    public boolean equals(isTableAvailable_result that) {
+    public boolean equals(isTableAvailableWithSplit_result that) {
       if (that == null)
         return false;
       if (this == that)
@@ -42266,7 +43430,7 @@ public class THBaseService {
     }
 
     @Override
-    public int compareTo(isTableAvailable_result other) {
+    public int compareTo(isTableAvailableWithSplit_result other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -42311,7 +43475,7 @@ public class THBaseService {
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("isTableAvailable_result(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("isTableAvailableWithSplit_result(");
       boolean first = true;
 
       sb.append("success:");
@@ -42352,15 +43516,15 @@ public class THBaseService {
       }
     }
 
-    private static class isTableAvailable_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public isTableAvailable_resultStandardScheme getScheme() {
-        return new isTableAvailable_resultStandardScheme();
+    private static class isTableAvailableWithSplit_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public isTableAvailableWithSplit_resultStandardScheme getScheme() {
+        return new isTableAvailableWithSplit_resultStandardScheme();
       }
     }
 
-    private static class isTableAvailable_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<isTableAvailable_result> {
+    private static class isTableAvailableWithSplit_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<isTableAvailableWithSplit_result> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, isTableAvailable_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, isTableAvailableWithSplit_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -42398,7 +43562,7 @@ public class THBaseService {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, isTableAvailable_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, isTableAvailableWithSplit_result struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -42418,16 +43582,16 @@ public class THBaseService {
 
     }
 
-    private static class isTableAvailable_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public isTableAvailable_resultTupleScheme getScheme() {
-        return new isTableAvailable_resultTupleScheme();
+    private static class isTableAvailableWithSplit_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public isTableAvailableWithSplit_resultTupleScheme getScheme() {
+        return new isTableAvailableWithSplit_resultTupleScheme();
       }
     }
 
-    private static class isTableAvailable_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<isTableAvailable_result> {
+    private static class isTableAvailableWithSplit_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<isTableAvailableWithSplit_result> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, isTableAvailable_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, isTableAvailableWithSplit_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet optionals = new java.util.BitSet();
         if (struct.isSetSuccess()) {
@@ -42446,7 +43610,7 @@ public class THBaseService {
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, isTableAvailable_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, isTableAvailableWithSplit_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
@@ -42466,34 +43630,34 @@ public class THBaseService {
     }
   }
 
-  public static class isTableAvailableWithSplit_args implements org.apache.thrift.TBase<isTableAvailableWithSplit_args, isTableAvailableWithSplit_args._Fields>, java.io.Serializable, Cloneable, Comparable<isTableAvailableWithSplit_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("isTableAvailableWithSplit_args");
+  public static class addColumnFamily_args implements org.apache.thrift.TBase<addColumnFamily_args, addColumnFamily_args._Fields>, java.io.Serializable, Cloneable, Comparable<addColumnFamily_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("addColumnFamily_args");
 
     private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRUCT, (short)1);
-    private static final org.apache.thrift.protocol.TField SPLIT_KEYS_FIELD_DESC = new org.apache.thrift.protocol.TField("splitKeys", org.apache.thrift.protocol.TType.LIST, (short)2);
+    private static final org.apache.thrift.protocol.TField COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("column", org.apache.thrift.protocol.TType.STRUCT, (short)2);
 
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new isTableAvailableWithSplit_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new isTableAvailableWithSplit_argsTupleSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new addColumnFamily_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new addColumnFamily_argsTupleSchemeFactory();
 
     /**
-     * the tablename to check
+     * the tablename to add column family to
      */
     public @org.apache.thrift.annotation.Nullable TTableName tableName; // required
     /**
-     * keys to check if the table has been created with all split keys
+     * column family descriptor of column family to be added
      */
-    public @org.apache.thrift.annotation.Nullable java.util.List<java.nio.ByteBuffer> splitKeys; // required
+    public @org.apache.thrift.annotation.Nullable TColumnFamilyDescriptor column; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
       /**
-       * the tablename to check
+       * the tablename to add column family to
        */
       TABLE_NAME((short)1, "tableName"),
       /**
-       * keys to check if the table has been created with all split keys
+       * column family descriptor of column family to be added
        */
-      SPLIT_KEYS((short)2, "splitKeys");
+      COLUMN((short)2, "column");
 
       private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
 
@@ -42511,8 +43675,8 @@ public class THBaseService {
         switch(fieldId) {
           case 1: // TABLE_NAME
             return TABLE_NAME;
-          case 2: // SPLIT_KEYS
-            return SPLIT_KEYS;
+          case 2: // COLUMN
+            return COLUMN;
           default:
             return null;
         }
@@ -42559,50 +43723,48 @@ public class THBaseService {
       java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
       tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TTableName.class)));
-      tmpMap.put(_Fields.SPLIT_KEYS, new org.apache.thrift.meta_data.FieldMetaData("splitKeys", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
-              new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING              , true))));
+      tmpMap.put(_Fields.COLUMN, new org.apache.thrift.meta_data.FieldMetaData("column", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TColumnFamilyDescriptor.class)));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(isTableAvailableWithSplit_args.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(addColumnFamily_args.class, metaDataMap);
     }
 
-    public isTableAvailableWithSplit_args() {
+    public addColumnFamily_args() {
     }
 
-    public isTableAvailableWithSplit_args(
+    public addColumnFamily_args(
       TTableName tableName,
-      java.util.List<java.nio.ByteBuffer> splitKeys)
+      TColumnFamilyDescriptor column)
     {
       this();
       this.tableName = tableName;
-      this.splitKeys = splitKeys;
+      this.column = column;
     }
 
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public isTableAvailableWithSplit_args(isTableAvailableWithSplit_args other) {
+    public addColumnFamily_args(addColumnFamily_args other) {
       if (other.isSetTableName()) {
         this.tableName = new TTableName(other.tableName);
       }
-      if (other.isSetSplitKeys()) {
-        java.util.List<java.nio.ByteBuffer> __this__splitKeys = new java.util.ArrayList<java.nio.ByteBuffer>(other.splitKeys);
-        this.splitKeys = __this__splitKeys;
+      if (other.isSetColumn()) {
+        this.column = new TColumnFamilyDescriptor(other.column);
       }
     }
 
-    public isTableAvailableWithSplit_args deepCopy() {
-      return new isTableAvailableWithSplit_args(this);
+    public addColumnFamily_args deepCopy() {
+      return new addColumnFamily_args(this);
     }
 
     @Override
     public void clear() {
       this.tableName = null;
-      this.splitKeys = null;
+      this.column = null;
     }
 
     /**
-     * the tablename to check
+     * the tablename to add column family to
      */
     @org.apache.thrift.annotation.Nullable
     public TTableName getTableName() {
@@ -42610,9 +43772,9 @@ public class THBaseService {
     }
 
     /**
-     * the tablename to check
+     * the tablename to add column family to
      */
-    public isTableAvailableWithSplit_args setTableName(@org.apache.thrift.annotation.Nullable TTableName tableName) {
+    public addColumnFamily_args setTableName(@org.apache.thrift.annotation.Nullable TTableName tableName) {
       this.tableName = tableName;
       return this;
     }
@@ -42632,50 +43794,34 @@ public class THBaseService {
       }
     }
 
-    public int getSplitKeysSize() {
-      return (this.splitKeys == null) ? 0 : this.splitKeys.size();
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.util.Iterator<java.nio.ByteBuffer> getSplitKeysIterator() {
-      return (this.splitKeys == null) ? null : this.splitKeys.iterator();
-    }
-
-    public void addToSplitKeys(java.nio.ByteBuffer elem) {
-      if (this.splitKeys == null) {
-        this.splitKeys = new java.util.ArrayList<java.nio.ByteBuffer>();
-      }
-      this.splitKeys.add(elem);
-    }
-
     /**
-     * keys to check if the table has been created with all split keys
+     * column family descriptor of column family to be added
      */
     @org.apache.thrift.annotation.Nullable
-    public java.util.List<java.nio.ByteBuffer> getSplitKeys() {
-      return this.splitKeys;
+    public TColumnFamilyDescriptor getColumn() {
+      return this.column;
     }
 
     /**
-     * keys to check if the table has been created with all split keys
+     * column family descriptor of column family to be added
      */
-    public isTableAvailableWithSplit_args setSplitKeys(@org.apache.thrift.annotation.Nullable java.util.List<java.nio.ByteBuffer> splitKeys) {
-      this.splitKeys = splitKeys;
+    public addColumnFamily_args setColumn(@org.apache.thrift.annotation.Nullable TColumnFamilyDescriptor column) {
+      this.column = column;
       return this;
     }
 
-    public void unsetSplitKeys() {
-      this.splitKeys = null;
+    public void unsetColumn() {
+      this.column = null;
     }
 
-    /** Returns true if field splitKeys is set (has been assigned a value) and false otherwise */
-    public boolean isSetSplitKeys() {
-      return this.splitKeys != null;
+    /** Returns true if field column is set (has been assigned a value) and false otherwise */
+    public boolean isSetColumn() {
+      return this.column != null;
     }
 
-    public void setSplitKeysIsSet(boolean value) {
+    public void setColumnIsSet(boolean value) {
       if (!value) {
-        this.splitKeys = null;
+        this.column = null;
       }
     }
 
@@ -42689,11 +43835,11 @@ public class THBaseService {
         }
         break;
 
-      case SPLIT_KEYS:
+      case COLUMN:
         if (value == null) {
-          unsetSplitKeys();
+          unsetColumn();
         } else {
-          setSplitKeys((java.util.List<java.nio.ByteBuffer>)value);
+          setColumn((TColumnFamilyDescriptor)value);
         }
         break;
 
@@ -42706,8 +43852,8 @@ public class THBaseService {
       case TABLE_NAME:
         return getTableName();
 
-      case SPLIT_KEYS:
-        return getSplitKeys();
+      case COLUMN:
+        return getColumn();
 
       }
       throw new java.lang.IllegalStateException();
@@ -42722,8 +43868,8 @@ public class THBaseService {
       switch (field) {
       case TABLE_NAME:
         return isSetTableName();
-      case SPLIT_KEYS:
-        return isSetSplitKeys();
+      case COLUMN:
+        return isSetColumn();
       }
       throw new java.lang.IllegalStateException();
     }
@@ -42732,12 +43878,12 @@ public class THBaseService {
     public boolean equals(java.lang.Object that) {
       if (that == null)
         return false;
-      if (that instanceof isTableAvailableWithSplit_args)
-        return this.equals((isTableAvailableWithSplit_args)that);
+      if (that instanceof addColumnFamily_args)
+        return this.equals((addColumnFamily_args)that);
       return false;
     }
 
-    public boolean equals(isTableAvailableWithSplit_args that) {
+    public boolean equals(addColumnFamily_args that) {
       if (that == null)
         return false;
       if (this == that)
@@ -42752,12 +43898,12 @@ public class THBaseService {
           return false;
       }
 
-      boolean this_present_splitKeys = true && this.isSetSplitKeys();
-      boolean that_present_splitKeys = true && that.isSetSplitKeys();
-      if (this_present_splitKeys || that_present_splitKeys) {
-        if (!(this_present_splitKeys && that_present_splitKeys))
+      boolean this_present_column = true && this.isSetColumn();
+      boolean that_present_column = true && that.isSetColumn();
+      if (this_present_column || that_present_column) {
+        if (!(this_present_column && that_present_column))
           return false;
-        if (!this.splitKeys.equals(that.splitKeys))
+        if (!this.column.equals(that.column))
           return false;
       }
 
@@ -42772,15 +43918,15 @@ public class THBaseService {
       if (isSetTableName())
         hashCode = hashCode * 8191 + tableName.hashCode();
 
-      hashCode = hashCode * 8191 + ((isSetSplitKeys()) ? 131071 : 524287);
-      if (isSetSplitKeys())
-        hashCode = hashCode * 8191 + splitKeys.hashCode();
+      hashCode = hashCode * 8191 + ((isSetColumn()) ? 131071 : 524287);
+      if (isSetColumn())
+        hashCode = hashCode * 8191 + column.hashCode();
 
       return hashCode;
     }
 
     @Override
-    public int compareTo(isTableAvailableWithSplit_args other) {
+    public int compareTo(addColumnFamily_args other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -42797,12 +43943,12 @@ public class THBaseService {
           return lastComparison;
         }
       }
-      lastComparison = java.lang.Boolean.valueOf(isSetSplitKeys()).compareTo(other.isSetSplitKeys());
+      lastComparison = java.lang.Boolean.valueOf(isSetColumn()).compareTo(other.isSetColumn());
       if (lastComparison != 0) {
         return lastComparison;
       }
-      if (isSetSplitKeys()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.splitKeys, other.splitKeys);
+      if (isSetColumn()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.column, other.column);
         if (lastComparison != 0) {
           return lastComparison;
         }
@@ -42825,7 +43971,7 @@ public class THBaseService {
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("isTableAvailableWithSplit_args(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("addColumnFamily_args(");
       boolean first = true;
 
       sb.append("tableName:");
@@ -42836,11 +43982,11 @@ public class THBaseService {
       }
       first = false;
       if (!first) sb.append(", ");
-      sb.append("splitKeys:");
-      if (this.splitKeys == null) {
+      sb.append("column:");
+      if (this.column == null) {
         sb.append("null");
       } else {
-        org.apache.thrift.TBaseHelper.toString(this.splitKeys, sb);
+        sb.append(this.column);
       }
       first = false;
       sb.append(")");
@@ -42852,10 +43998,16 @@ public class THBaseService {
       if (tableName == null) {
         throw new org.apache.thrift.protocol.TProtocolException("Required field 'tableName' was not present! Struct: " + toString());
       }
+      if (column == null) {
+        throw new org.apache.thrift.protocol.TProtocolException("Required field 'column' was not present! Struct: " + toString());
+      }
       // check for sub-struct validity
       if (tableName != null) {
         tableName.validate();
       }
+      if (column != null) {
+        column.validate();
+      }
     }
 
     private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
@@ -42874,15 +44026,15 @@ public class THBaseService {
       }
     }
 
-    private static class isTableAvailableWithSplit_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public isTableAvailableWithSplit_argsStandardScheme getScheme() {
-        return new isTableAvailableWithSplit_argsStandardScheme();
+    private static class addColumnFamily_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public addColumnFamily_argsStandardScheme getScheme() {
+        return new addColumnFamily_argsStandardScheme();
       }
     }
 
-    private static class isTableAvailableWithSplit_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<isTableAvailableWithSplit_args> {
+    private static class addColumnFamily_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<addColumnFamily_args> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, isTableAvailableWithSplit_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, addColumnFamily_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -42901,20 +44053,11 @@ public class THBaseService {
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
               break;
-            case 2: // SPLIT_KEYS
-              if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
-                {
-                  org.apache.thrift.protocol.TList _list326 = iprot.readListBegin();
-                  struct.splitKeys = new java.util.ArrayList<java.nio.ByteBuffer>(_list326.size);
-                  @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem327;
-                  for (int _i328 = 0; _i328 < _list326.size; ++_i328)
-                  {
-                    _elem327 = iprot.readBinary();
-                    struct.splitKeys.add(_elem327);
-                  }
-                  iprot.readListEnd();
-                }
-                struct.setSplitKeysIsSet(true);
+            case 2: // COLUMN
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.column = new TColumnFamilyDescriptor();
+                struct.column.read(iprot);
+                struct.setColumnIsSet(true);
               } else { 
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
@@ -42930,7 +44073,7 @@ public class THBaseService {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, isTableAvailableWithSplit_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, addColumnFamily_args struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -42939,16 +44082,9 @@ public class THBaseService {
           struct.tableName.write(oprot);
           oprot.writeFieldEnd();
         }
-        if (struct.splitKeys != null) {
-          oprot.writeFieldBegin(SPLIT_KEYS_FIELD_DESC);
-          {
-            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.splitKeys.size()));
-            for (java.nio.ByteBuffer _iter329 : struct.splitKeys)
-            {
-              oprot.writeBinary(_iter329);
-            }
-            oprot.writeListEnd();
-          }
+        if (struct.column != null) {
+          oprot.writeFieldBegin(COLUMN_FIELD_DESC);
+          struct.column.write(oprot);
           oprot.writeFieldEnd();
         }
         oprot.writeFieldStop();
@@ -42957,54 +44093,30 @@ public class THBaseService {
 
     }
 
-    private static class isTableAvailableWithSplit_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public isTableAvailableWithSplit_argsTupleScheme getScheme() {
-        return new isTableAvailableWithSplit_argsTupleScheme();
+    private static class addColumnFamily_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public addColumnFamily_argsTupleScheme getScheme() {
+        return new addColumnFamily_argsTupleScheme();
       }
     }
 
-    private static class isTableAvailableWithSplit_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<isTableAvailableWithSplit_args> {
+    private static class addColumnFamily_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<addColumnFamily_args> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, isTableAvailableWithSplit_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, addColumnFamily_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         struct.tableName.write(oprot);
-        java.util.BitSet optionals = new java.util.BitSet();
-        if (struct.isSetSplitKeys()) {
-          optionals.set(0);
-        }
-        oprot.writeBitSet(optionals, 1);
-        if (struct.isSetSplitKeys()) {
-          {
-            oprot.writeI32(struct.splitKeys.size());
-            for (java.nio.ByteBuffer _iter330 : struct.splitKeys)
-            {
-              oprot.writeBinary(_iter330);
-            }
-          }
-        }
+        struct.column.write(oprot);
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, isTableAvailableWithSplit_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, addColumnFamily_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         struct.tableName = new TTableName();
         struct.tableName.read(iprot);
         struct.setTableNameIsSet(true);
-        java.util.BitSet incoming = iprot.readBitSet(1);
-        if (incoming.get(0)) {
-          {
-            org.apache.thrift.protocol.TList _list331 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.splitKeys = new java.util.ArrayList<java.nio.ByteBuffer>(_list331.size);
-            @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem332;
-            for (int _i333 = 0; _i333 < _list331.size; ++_i333)
-            {
-              _elem332 = iprot.readBinary();
-              struct.splitKeys.add(_elem332);
-            }
-          }
-          struct.setSplitKeysIsSet(true);
-        }
+        struct.column = new TColumnFamilyDescriptor();
+        struct.column.read(iprot);
+        struct.setColumnIsSet(true);
       }
     }
 
@@ -43013,21 +44125,18 @@ public class THBaseService {
     }
   }
 
-  public static class isTableAvailableWithSplit_result implements org.apache.thrift.TBase<isTableAvailableWithSplit_result, isTableAvailableWithSplit_result._Fields>, java.io.Serializable, Cloneable, Comparable<isTableAvailableWithSplit_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("isTableAvailableWithSplit_result");
+  public static class addColumnFamily_result implements org.apache.thrift.TBase<addColumnFamily_result, addColumnFamily_result._Fields>, java.io.Serializable, Cloneable, Comparable<addColumnFamily_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("addColumnFamily_result");
 
-    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.BOOL, (short)0);
     private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new isTableAvailableWithSplit_resultStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new isTableAvailableWithSplit_resultTupleSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new addColumnFamily_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new addColumnFamily_resultTupleSchemeFactory();
 
-    public boolean success; // required
     public @org.apache.thrift.annotation.Nullable TIOError io; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-      SUCCESS((short)0, "success"),
       IO((short)1, "io");
 
       private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
@@ -43044,8 +44153,6 @@ public class THBaseService {
       @org.apache.thrift.annotation.Nullable
       public static _Fields findByThriftId(int fieldId) {
         switch(fieldId) {
-          case 0: // SUCCESS
-            return SUCCESS;
           case 1: // IO
             return IO;
           default:
@@ -43089,83 +44196,49 @@ public class THBaseService {
     }
 
     // isset id assignments
-    private static final int __SUCCESS_ISSET_ID = 0;
-    private byte __isset_bitfield = 0;
     public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
       tmpMap.put(_Fields.IO, new org.apache.thrift.meta_data.FieldMetaData("io", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TIOError.class)));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(isTableAvailableWithSplit_result.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(addColumnFamily_result.class, metaDataMap);
     }
 
-    public isTableAvailableWithSplit_result() {
+    public addColumnFamily_result() {
     }
 
-    public isTableAvailableWithSplit_result(
-      boolean success,
+    public addColumnFamily_result(
       TIOError io)
     {
       this();
-      this.success = success;
-      setSuccessIsSet(true);
       this.io = io;
     }
 
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public isTableAvailableWithSplit_result(isTableAvailableWithSplit_result other) {
-      __isset_bitfield = other.__isset_bitfield;
-      this.success = other.success;
+    public addColumnFamily_result(addColumnFamily_result other) {
       if (other.isSetIo()) {
         this.io = new TIOError(other.io);
       }
     }
 
-    public isTableAvailableWithSplit_result deepCopy() {
-      return new isTableAvailableWithSplit_result(this);
+    public addColumnFamily_result deepCopy() {
+      return new addColumnFamily_result(this);
     }
 
     @Override
     public void clear() {
-      setSuccessIsSet(false);
-      this.success = false;
       this.io = null;
     }
 
-    public boolean isSuccess() {
-      return this.success;
-    }
-
-    public isTableAvailableWithSplit_result setSuccess(boolean success) {
-      this.success = success;
-      setSuccessIsSet(true);
-      return this;
-    }
-
-    public void unsetSuccess() {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __SUCCESS_ISSET_ID);
-    }
-
-    /** Returns true if field success is set (has been assigned a value) and false otherwise */
-    public boolean isSetSuccess() {
-      return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __SUCCESS_ISSET_ID);
-    }
-
-    public void setSuccessIsSet(boolean value) {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __SUCCESS_ISSET_ID, value);
-    }
-
     @org.apache.thrift.annotation.Nullable
     public TIOError getIo() {
       return this.io;
     }
 
-    public isTableAvailableWithSplit_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) {
+    public addColumnFamily_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) {
       this.io = io;
       return this;
     }
@@ -43187,14 +44260,6 @@ public class THBaseService {
 
     public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
       switch (field) {
-      case SUCCESS:
-        if (value == null) {
-          unsetSuccess();
-        } else {
-          setSuccess((java.lang.Boolean)value);
-        }
-        break;
-
       case IO:
         if (value == null) {
           unsetIo();
@@ -43209,9 +44274,6 @@ public class THBaseService {
     @org.apache.thrift.annotation.Nullable
     public java.lang.Object getFieldValue(_Fields field) {
       switch (field) {
-      case SUCCESS:
-        return isSuccess();
-
       case IO:
         return getIo();
 
@@ -43226,8 +44288,6 @@ public class THBaseService {
       }
 
       switch (field) {
-      case SUCCESS:
-        return isSetSuccess();
       case IO:
         return isSetIo();
       }
@@ -43238,26 +44298,17 @@ public class THBaseService {
     public boolean equals(java.lang.Object that) {
       if (that == null)
         return false;
-      if (that instanceof isTableAvailableWithSplit_result)
-        return this.equals((isTableAvailableWithSplit_result)that);
+      if (that instanceof addColumnFamily_result)
+        return this.equals((addColumnFamily_result)that);
       return false;
     }
 
-    public boolean equals(isTableAvailableWithSplit_result that) {
+    public boolean equals(addColumnFamily_result that) {
       if (that == null)
         return false;
       if (this == that)
         return true;
 
-      boolean this_present_success = true;
-      boolean that_present_success = true;
-      if (this_present_success || that_present_success) {
-        if (!(this_present_success && that_present_success))
-          return false;
-        if (this.success != that.success)
-          return false;
-      }
-
       boolean this_present_io = true && this.isSetIo();
       boolean that_present_io = true && that.isSetIo();
       if (this_present_io || that_present_io) {
@@ -43274,8 +44325,6 @@ public class THBaseService {
     public int hashCode() {
       int hashCode = 1;
 
-      hashCode = hashCode * 8191 + ((success) ? 131071 : 524287);
-
       hashCode = hashCode * 8191 + ((isSetIo()) ? 131071 : 524287);
       if (isSetIo())
         hashCode = hashCode * 8191 + io.hashCode();
@@ -43284,23 +44333,13 @@ public class THBaseService {
     }
 
     @Override
-    public int compareTo(isTableAvailableWithSplit_result other) {
+    public int compareTo(addColumnFamily_result other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
 
       int lastComparison = 0;
 
-      lastComparison = java.lang.Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetSuccess()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
       lastComparison = java.lang.Boolean.valueOf(isSetIo()).compareTo(other.isSetIo());
       if (lastComparison != 0) {
         return lastComparison;
@@ -43329,13 +44368,9 @@ public class THBaseService {
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("isTableAvailableWithSplit_result(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("addColumnFamily_result(");
       boolean first = true;
 
-      sb.append("success:");
-      sb.append(this.success);
-      first = false;
-      if (!first) sb.append(", ");
       sb.append("io:");
       if (this.io == null) {
         sb.append("null");
@@ -43362,23 +44397,21 @@ public class THBaseService {
 
     private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
       try {
-        // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
-        __isset_bitfield = 0;
         read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
       } catch (org.apache.thrift.TException te) {
         throw new java.io.IOException(te);
       }
     }
 
-    private static class isTableAvailableWithSplit_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public isTableAvailableWithSplit_resultStandardScheme getScheme() {
-        return new isTableAvailableWithSplit_resultStandardScheme();
+    private static class addColumnFamily_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public addColumnFamily_resultStandardScheme getScheme() {
+        return new addColumnFamily_resultStandardScheme();
       }
     }
 
-    private static class isTableAvailableWithSplit_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<isTableAvailableWithSplit_result> {
+    private static class addColumnFamily_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<addColumnFamily_result> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, isTableAvailableWithSplit_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, addColumnFamily_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -43388,14 +44421,6 @@ public class THBaseService {
             break;
           }
           switch (schemeField.id) {
-            case 0: // SUCCESS
-              if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
-                struct.success = iprot.readBool();
-                struct.setSuccessIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
             case 1: // IO
               if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
                 struct.io = new TIOError();
@@ -43416,15 +44441,10 @@ public class THBaseService {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, isTableAvailableWithSplit_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, addColumnFamily_result struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
-        if (struct.isSetSuccess()) {
-          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
-          oprot.writeBool(struct.success);
-          oprot.writeFieldEnd();
-        }
         if (struct.io != null) {
           oprot.writeFieldBegin(IO_FIELD_DESC);
           struct.io.write(oprot);
@@ -43436,42 +44456,32 @@ public class THBaseService {
 
     }
 
-    private static class isTableAvailableWithSplit_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public isTableAvailableWithSplit_resultTupleScheme getScheme() {
-        return new isTableAvailableWithSplit_resultTupleScheme();
+    private static class addColumnFamily_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public addColumnFamily_resultTupleScheme getScheme() {
+        return new addColumnFamily_resultTupleScheme();
       }
     }
 
-    private static class isTableAvailableWithSplit_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<isTableAvailableWithSplit_result> {
+    private static class addColumnFamily_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<addColumnFamily_result> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, isTableAvailableWithSplit_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, addColumnFamily_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet optionals = new java.util.BitSet();
-        if (struct.isSetSuccess()) {
-          optionals.set(0);
-        }
         if (struct.isSetIo()) {
-          optionals.set(1);
-        }
-        oprot.writeBitSet(optionals, 2);
-        if (struct.isSetSuccess()) {
-          oprot.writeBool(struct.success);
+          optionals.set(0);
         }
+        oprot.writeBitSet(optionals, 1);
         if (struct.isSetIo()) {
           struct.io.write(oprot);
         }
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, isTableAvailableWithSplit_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, addColumnFamily_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
-        java.util.BitSet incoming = iprot.readBitSet(2);
+        java.util.BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
-          struct.success = iprot.readBool();
-          struct.setSuccessIsSet(true);
-        }
-        if (incoming.get(1)) {
           struct.io = new TIOError();
           struct.io.read(iprot);
           struct.setIoIsSet(true);
@@ -43484,32 +44494,32 @@ public class THBaseService {
     }
   }
 
-  public static class addColumnFamily_args implements org.apache.thrift.TBase<addColumnFamily_args, addColumnFamily_args._Fields>, java.io.Serializable, Cloneable, Comparable<addColumnFamily_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("addColumnFamily_args");
+  public static class deleteColumnFamily_args implements org.apache.thrift.TBase<deleteColumnFamily_args, deleteColumnFamily_args._Fields>, java.io.Serializable, Cloneable, Comparable<deleteColumnFamily_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("deleteColumnFamily_args");
 
     private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRUCT, (short)1);
-    private static final org.apache.thrift.protocol.TField COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("column", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+    private static final org.apache.thrift.protocol.TField COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("column", org.apache.thrift.protocol.TType.STRING, (short)2);
 
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new addColumnFamily_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new addColumnFamily_argsTupleSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new deleteColumnFamily_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new deleteColumnFamily_argsTupleSchemeFactory();
 
     /**
-     * the tablename to add column family to
+     * the tablename to delete column family from
      */
     public @org.apache.thrift.annotation.Nullable TTableName tableName; // required
     /**
-     * column family descriptor of column family to be added
+     * name of column family to be deleted
      */
-    public @org.apache.thrift.annotation.Nullable TColumnFamilyDescriptor column; // required
+    public @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer column; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
       /**
-       * the tablename to add column family to
+       * the tablename to delete column family from
        */
       TABLE_NAME((short)1, "tableName"),
       /**
-       * column family descriptor of column family to be added
+       * name of column family to be deleted
        */
       COLUMN((short)2, "column");
 
@@ -43578,37 +44588,37 @@ public class THBaseService {
       tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TTableName.class)));
       tmpMap.put(_Fields.COLUMN, new org.apache.thrift.meta_data.FieldMetaData("column", org.apache.thrift.TFieldRequirementType.REQUIRED, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TColumnFamilyDescriptor.class)));
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING          , true)));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(addColumnFamily_args.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(deleteColumnFamily_args.class, metaDataMap);
     }
 
-    public addColumnFamily_args() {
+    public deleteColumnFamily_args() {
     }
 
-    public addColumnFamily_args(
+    public deleteColumnFamily_args(
       TTableName tableName,
-      TColumnFamilyDescriptor column)
+      java.nio.ByteBuffer column)
     {
       this();
       this.tableName = tableName;
-      this.column = column;
+      this.column = org.apache.thrift.TBaseHelper.copyBinary(column);
     }
 
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public addColumnFamily_args(addColumnFamily_args other) {
+    public deleteColumnFamily_args(deleteColumnFamily_args other) {
       if (other.isSetTableName()) {
         this.tableName = new TTableName(other.tableName);
       }
       if (other.isSetColumn()) {
-        this.column = new TColumnFamilyDescriptor(other.column);
+        this.column = org.apache.thrift.TBaseHelper.copyBinary(other.column);
       }
     }
 
-    public addColumnFamily_args deepCopy() {
-      return new addColumnFamily_args(this);
+    public deleteColumnFamily_args deepCopy() {
+      return new deleteColumnFamily_args(this);
     }
 
     @Override
@@ -43618,7 +44628,7 @@ public class THBaseService {
     }
 
     /**
-     * the tablename to add column family to
+     * the tablename to delete column family from
      */
     @org.apache.thrift.annotation.Nullable
     public TTableName getTableName() {
@@ -43626,9 +44636,9 @@ public class THBaseService {
     }
 
     /**
-     * the tablename to add column family to
+     * the tablename to delete column family from
      */
-    public addColumnFamily_args setTableName(@org.apache.thrift.annotation.Nullable TTableName tableName) {
+    public deleteColumnFamily_args setTableName(@org.apache.thrift.annotation.Nullable TTableName tableName) {
       this.tableName = tableName;
       return this;
     }
@@ -43649,18 +44659,27 @@ public class THBaseService {
     }
 
     /**
-     * column family descriptor of column family to be added
+     * name of column family to be deleted
      */
-    @org.apache.thrift.annotation.Nullable
-    public TColumnFamilyDescriptor getColumn() {
-      return this.column;
+    public byte[] getColumn() {
+      setColumn(org.apache.thrift.TBaseHelper.rightSize(column));
+      return column == null ? null : column.array();
+    }
+
+    public java.nio.ByteBuffer bufferForColumn() {
+      return org.apache.thrift.TBaseHelper.copyBinary(column);
     }
 
     /**
-     * column family descriptor of column family to be added
+     * name of column family to be deleted
      */
-    public addColumnFamily_args setColumn(@org.apache.thrift.annotation.Nullable TColumnFamilyDescriptor column) {
-      this.column = column;
+    public deleteColumnFamily_args setColumn(byte[] column) {
+      this.column = column == null ? (java.nio.ByteBuffer)null     : java.nio.ByteBuffer.wrap(column.clone());
+      return this;
+    }
+
+    public deleteColumnFamily_args setColumn(@org.apache.thrift.annotation.Nullable java.nio.ByteBuffer column) {
+      this.column = org.apache.thrift.TBaseHelper.copyBinary(column);
       return this;
     }
 
@@ -43693,7 +44712,11 @@ public class THBaseService {
         if (value == null) {
           unsetColumn();
         } else {
-          setColumn((TColumnFamilyDescriptor)value);
+          if (value instanceof byte[]) {
+            setColumn((byte[])value);
+          } else {
+            setColumn((java.nio.ByteBuffer)value);
+          }
         }
         break;
 
@@ -43732,12 +44755,12 @@ public class THBaseService {
     public boolean equals(java.lang.Object that) {
       if (that == null)
         return false;
-      if (that instanceof addColumnFamily_args)
-        return this.equals((addColumnFamily_args)that);
+      if (that instanceof deleteColumnFamily_args)
+        return this.equals((deleteColumnFamily_args)that);
       return false;
     }
 
-    public boolean equals(addColumnFamily_args that) {
+    public boolean equals(deleteColumnFamily_args that) {
       if (that == null)
         return false;
       if (this == that)
@@ -43780,7 +44803,7 @@ public class THBaseService {
     }
 
     @Override
-    public int compareTo(addColumnFamily_args other) {
+    public int compareTo(deleteColumnFamily_args other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -43825,7 +44848,7 @@ public class THBaseService {
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("addColumnFamily_args(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("deleteColumnFamily_args(");
       boolean first = true;
 
       sb.append("tableName:");
@@ -43840,7 +44863,7 @@ public class THBaseService {
       if (this.column == null) {
         sb.append("null");
       } else {
-        sb.append(this.column);
+        org.apache.thrift.TBaseHelper.toString(this.column, sb);
       }
       first = false;
       sb.append(")");
@@ -43859,9 +44882,6 @@ public class THBaseService {
       if (tableName != null) {
         tableName.validate();
       }
-      if (column != null) {
-        column.validate();
-      }
     }
 
     private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
@@ -43880,15 +44900,15 @@ public class THBaseService {
       }
     }
 
-    private static class addColumnFamily_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public addColumnFamily_argsStandardScheme getScheme() {
-        return new addColumnFamily_argsStandardScheme();
+    private static class deleteColumnFamily_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public deleteColumnFamily_argsStandardScheme getScheme() {
+        return new deleteColumnFamily_argsStandardScheme();
       }
     }
 
-    private static class addColumnFamily_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<addColumnFamily_args> {
+    private static class deleteColumnFamily_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<deleteColumnFamily_args> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, addColumnFamily_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, deleteColumnFamily_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -43908,9 +44928,8 @@ public class THBaseService {
               }
               break;
             case 2: // COLUMN
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.column = new TColumnFamilyDescriptor();
-                struct.column.read(iprot);
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.column = iprot.readBinary();
                 struct.setColumnIsSet(true);
               } else { 
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
@@ -43927,7 +44946,7 @@ public class THBaseService {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, addColumnFamily_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, deleteColumnFamily_args struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -43938,7 +44957,7 @@ public class THBaseService {
         }
         if (struct.column != null) {
           oprot.writeFieldBegin(COLUMN_FIELD_DESC);
-          struct.column.write(oprot);
+          oprot.writeBinary(struct.column);
           oprot.writeFieldEnd();
         }
         oprot.writeFieldStop();
@@ -43947,29 +44966,28 @@ public class THBaseService {
 
     }
 
-    private static class addColumnFamily_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public addColumnFamily_argsTupleScheme getScheme() {
-        return new addColumnFamily_argsTupleScheme();
+    private static class deleteColumnFamily_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public deleteColumnFamily_argsTupleScheme getScheme() {
+        return new deleteColumnFamily_argsTupleScheme();
       }
     }
 
-    private static class addColumnFamily_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<addColumnFamily_args> {
+    private static class deleteColumnFamily_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<deleteColumnFamily_args> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, addColumnFamily_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, deleteColumnFamily_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         struct.tableName.write(oprot);
-        struct.column.write(oprot);
+        oprot.writeBinary(struct.column);
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, addColumnFamily_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, deleteColumnFamily_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         struct.tableName = new TTableName();
         struct.tableName.read(iprot);
         struct.setTableNameIsSet(true);
-        struct.column = new TColumnFamilyDescriptor();
-        struct.column.read(iprot);
+        struct.column = iprot.readBinary();
         struct.setColumnIsSet(true);
       }
     }
@@ -43979,13 +44997,13 @@ public class THBaseService {
     }
   }
 
-  public static class addColumnFamily_result implements org.apache.thrift.TBase<addColumnFamily_result, addColumnFamily_result._Fields>, java.io.Serializable, Cloneable, Comparable<addColumnFamily_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("addColumnFamily_result");
+  public static class deleteColumnFamily_result implements org.apache.thrift.TBase<deleteColumnFamily_result, deleteColumnFamily_result._Fields>, java.io.Serializable, Cloneable, Comparable<deleteColumnFamily_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("deleteColumnFamily_result");
 
     private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new addColumnFamily_resultStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new addColumnFamily_resultTupleSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new deleteColumnFamily_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new deleteColumnFamily_resultTupleSchemeFactory();
 
     public @org.apache.thrift.annotation.Nullable TIOError io; // required
 
@@ -44056,13 +45074,13 @@ public class THBaseService {
       tmpMap.put(_Fields.IO, new org.apache.thrift.meta_data.FieldMetaData("io", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TIOError.class)));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(addColumnFamily_result.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(deleteColumnFamily_result.class, metaDataMap);
     }
 
-    public addColumnFamily_result() {
+    public deleteColumnFamily_result() {
     }
 
-    public addColumnFamily_result(
+    public deleteColumnFamily_result(
       TIOError io)
     {
       this();
@@ -44072,14 +45090,14 @@ public class THBaseService {
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public addColumnFamily_result(addColumnFamily_result other) {
+    public deleteColumnFamily_result(deleteColumnFamily_result other) {
       if (other.isSetIo()) {
         this.io = new TIOError(other.io);
       }
     }
 
-    public addColumnFamily_result deepCopy() {
-      return new addColumnFamily_result(this);
+    public deleteColumnFamily_result deepCopy() {
+      return new deleteColumnFamily_result(this);
     }
 
     @Override
@@ -44092,7 +45110,7 @@ public class THBaseService {
       return this.io;
     }
 
-    public addColumnFamily_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) {
+    public deleteColumnFamily_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) {
       this.io = io;
       return this;
     }
@@ -44152,12 +45170,12 @@ public class THBaseService {
     public boolean equals(java.lang.Object that) {
       if (that == null)
         return false;
-      if (that instanceof addColumnFamily_result)
-        return this.equals((addColumnFamily_result)that);
+      if (that instanceof deleteColumnFamily_result)
+        return this.equals((deleteColumnFamily_result)that);
       return false;
     }
 
-    public boolean equals(addColumnFamily_result that) {
+    public boolean equals(deleteColumnFamily_result that) {
       if (that == null)
         return false;
       if (this == that)
@@ -44187,7 +45205,7 @@ public class THBaseService {
     }
 
     @Override
-    public int compareTo(addColumnFamily_result other) {
+    public int compareTo(deleteColumnFamily_result other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -44222,7 +45240,7 @@ public class THBaseService {
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("addColumnFamily_result(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("deleteColumnFamily_result(");
       boolean first = true;
 
       sb.append("io:");
@@ -44257,15 +45275,15 @@ public class THBaseService {
       }
     }
 
-    private static class addColumnFamily_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public addColumnFamily_resultStandardScheme getScheme() {
-        return new addColumnFamily_resultStandardScheme();
+    private static class deleteColumnFamily_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public deleteColumnFamily_resultStandardScheme getScheme() {
+        return new deleteColumnFamily_resultStandardScheme();
       }
     }
 
-    private static class addColumnFamily_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<addColumnFamily_result> {
+    private static class deleteColumnFamily_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<deleteColumnFamily_result> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, addColumnFamily_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, deleteColumnFamily_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -44295,7 +45313,7 @@ public class THBaseService {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, addColumnFamily_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, deleteColumnFamily_result struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -44310,16 +45328,16 @@ public class THBaseService {
 
     }
 
-    private static class addColumnFamily_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public addColumnFamily_resultTupleScheme getScheme() {
-        return new addColumnFamily_resultTupleScheme();
+    private static class deleteColumnFamily_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public deleteColumnFamily_resultTupleScheme getScheme() {
+        return new deleteColumnFamily_resultTupleScheme();
       }
     }
 
-    private static class addColumnFamily_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<addColumnFamily_result> {
+    private static class deleteColumnFamily_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<deleteColumnFamily_result> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, addColumnFamily_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, deleteColumnFamily_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet optionals = new java.util.BitSet();
         if (struct.isSetIo()) {
@@ -44332,7 +45350,7 @@ public class THBaseService {
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, addColumnFamily_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, deleteColumnFamily_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
@@ -44348,32 +45366,32 @@ public class THBaseService {
     }
   }
 
-  public static class deleteColumnFamily_args implements org.apache.thrift.TBase<deleteColumnFamily_args, deleteColumnFamily_args._Fields>, java.io.Serializable, Cloneable, Comparable<deleteColumnFamily_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("deleteColumnFamily_args");
+  public static class modifyColumnFamily_args implements org.apache.thrift.TBase<modifyColumnFamily_args, modifyColumnFamily_args._Fields>, java.io.Serializable, Cloneable, Comparable<modifyColumnFamily_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("modifyColumnFamily_args");
 
     private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRUCT, (short)1);
-    private static final org.apache.thrift.protocol.TField COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("column", org.apache.thrift.protocol.TType.STRING, (short)2);
+    private static final org.apache.thrift.protocol.TField COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("column", org.apache.thrift.protocol.TType.STRUCT, (short)2);
 
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new deleteColumnFamily_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new deleteColumnFamily_argsTupleSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new modifyColumnFamily_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new modifyColumnFamily_argsTupleSchemeFactory();
 
     /**
-     * the tablename to delete column family from
+     * the tablename to modify column family
      */
     public @org.apache.thrift.annotation.Nullable TTableName tableName; // required
     /**
-     * name of column family to be deleted
+     * column family descriptor of column family to be modified
      */
-    public @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer column; // required
+    public @org.apache.thrift.annotation.Nullable TColumnFamilyDescriptor column; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
       /**
-       * the tablename to delete column family from
+       * the tablename to modify column family
        */
       TABLE_NAME((short)1, "tableName"),
       /**
-       * name of column family to be deleted
+       * column family descriptor of column family to be modified
        */
       COLUMN((short)2, "column");
 
@@ -44442,37 +45460,37 @@ public class THBaseService {
       tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TTableName.class)));
       tmpMap.put(_Fields.COLUMN, new org.apache.thrift.meta_data.FieldMetaData("column", org.apache.thrift.TFieldRequirementType.REQUIRED, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING          , true)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TColumnFamilyDescriptor.class)));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(deleteColumnFamily_args.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(modifyColumnFamily_args.class, metaDataMap);
     }
 
-    public deleteColumnFamily_args() {
+    public modifyColumnFamily_args() {
     }
 
-    public deleteColumnFamily_args(
+    public modifyColumnFamily_args(
       TTableName tableName,
-      java.nio.ByteBuffer column)
+      TColumnFamilyDescriptor column)
     {
       this();
       this.tableName = tableName;
-      this.column = org.apache.thrift.TBaseHelper.copyBinary(column);
+      this.column = column;
     }
 
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public deleteColumnFamily_args(deleteColumnFamily_args other) {
+    public modifyColumnFamily_args(modifyColumnFamily_args other) {
       if (other.isSetTableName()) {
         this.tableName = new TTableName(other.tableName);
       }
       if (other.isSetColumn()) {
-        this.column = org.apache.thrift.TBaseHelper.copyBinary(other.column);
+        this.column = new TColumnFamilyDescriptor(other.column);
       }
     }
 
-    public deleteColumnFamily_args deepCopy() {
-      return new deleteColumnFamily_args(this);
+    public modifyColumnFamily_args deepCopy() {
+      return new modifyColumnFamily_args(this);
     }
 
     @Override
@@ -44482,7 +45500,7 @@ public class THBaseService {
     }
 
     /**
-     * the tablename to delete column family from
+     * the tablename to modify column family
      */
     @org.apache.thrift.annotation.Nullable
     public TTableName getTableName() {
@@ -44490,9 +45508,9 @@ public class THBaseService {
     }
 
     /**
-     * the tablename to delete column family from
+     * the tablename to modify column family
      */
-    public deleteColumnFamily_args setTableName(@org.apache.thrift.annotation.Nullable TTableName tableName) {
+    public modifyColumnFamily_args setTableName(@org.apache.thrift.annotation.Nullable TTableName tableName) {
       this.tableName = tableName;
       return this;
     }
@@ -44513,27 +45531,18 @@ public class THBaseService {
     }
 
     /**
-     * name of column family to be deleted
+     * column family descriptor of column family to be modified
      */
-    public byte[] getColumn() {
-      setColumn(org.apache.thrift.TBaseHelper.rightSize(column));
-      return column == null ? null : column.array();
-    }
-
-    public java.nio.ByteBuffer bufferForColumn() {
-      return org.apache.thrift.TBaseHelper.copyBinary(column);
+    @org.apache.thrift.annotation.Nullable
+    public TColumnFamilyDescriptor getColumn() {
+      return this.column;
     }
 
     /**
-     * name of column family to be deleted
+     * column family descriptor of column family to be modified
      */
-    public deleteColumnFamily_args setColumn(byte[] column) {
-      this.column = column == null ? (java.nio.ByteBuffer)null     : java.nio.ByteBuffer.wrap(column.clone());
-      return this;
-    }
-
-    public deleteColumnFamily_args setColumn(@org.apache.thrift.annotation.Nullable java.nio.ByteBuffer column) {
-      this.column = org.apache.thrift.TBaseHelper.copyBinary(column);
+    public modifyColumnFamily_args setColumn(@org.apache.thrift.annotation.Nullable TColumnFamilyDescriptor column) {
+      this.column = column;
       return this;
     }
 
@@ -44566,11 +45575,7 @@ public class THBaseService {
         if (value == null) {
           unsetColumn();
         } else {
-          if (value instanceof byte[]) {
-            setColumn((byte[])value);
-          } else {
-            setColumn((java.nio.ByteBuffer)value);
-          }
+          setColumn((TColumnFamilyDescriptor)value);
         }
         break;
 
@@ -44609,12 +45614,12 @@ public class THBaseService {
     public boolean equals(java.lang.Object that) {
       if (that == null)
         return false;
-      if (that instanceof deleteColumnFamily_args)
-        return this.equals((deleteColumnFamily_args)that);
+      if (that instanceof modifyColumnFamily_args)
+        return this.equals((modifyColumnFamily_args)that);
       return false;
     }
 
-    public boolean equals(deleteColumnFamily_args that) {
+    public boolean equals(modifyColumnFamily_args that) {
       if (that == null)
         return false;
       if (this == that)
@@ -44657,7 +45662,7 @@ public class THBaseService {
     }
 
     @Override
-    public int compareTo(deleteColumnFamily_args other) {
+    public int compareTo(modifyColumnFamily_args other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -44702,7 +45707,7 @@ public class THBaseService {
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("deleteColumnFamily_args(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("modifyColumnFamily_args(");
       boolean first = true;
 
       sb.append("tableName:");
@@ -44717,7 +45722,7 @@ public class THBaseService {
       if (this.column == null) {
         sb.append("null");
       } else {
-        org.apache.thrift.TBaseHelper.toString(this.column, sb);
+        sb.append(this.column);
       }
       first = false;
       sb.append(")");
@@ -44736,6 +45741,9 @@ public class THBaseService {
       if (tableName != null) {
         tableName.validate();
       }
+      if (column != null) {
+        column.validate();
+      }
     }
 
     private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
@@ -44754,15 +45762,15 @@ public class THBaseService {
       }
     }
 
-    private static class deleteColumnFamily_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public deleteColumnFamily_argsStandardScheme getScheme() {
-        return new deleteColumnFamily_argsStandardScheme();
+    private static class modifyColumnFamily_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public modifyColumnFamily_argsStandardScheme getScheme() {
+        return new modifyColumnFamily_argsStandardScheme();
       }
     }
 
-    private static class deleteColumnFamily_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<deleteColumnFamily_args> {
+    private static class modifyColumnFamily_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<modifyColumnFamily_args> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, deleteColumnFamily_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, modifyColumnFamily_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -44782,8 +45790,9 @@ public class THBaseService {
               }
               break;
             case 2: // COLUMN
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
-                struct.column = iprot.readBinary();
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.column = new TColumnFamilyDescriptor();
+                struct.column.read(iprot);
                 struct.setColumnIsSet(true);
               } else { 
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
@@ -44800,7 +45809,7 @@ public class THBaseService {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, deleteColumnFamily_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, modifyColumnFamily_args struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -44811,7 +45820,7 @@ public class THBaseService {
         }
         if (struct.column != null) {
           oprot.writeFieldBegin(COLUMN_FIELD_DESC);
-          oprot.writeBinary(struct.column);
+          struct.column.write(oprot);
           oprot.writeFieldEnd();
         }
         oprot.writeFieldStop();
@@ -44820,28 +45829,29 @@ public class THBaseService {
 
     }
 
-    private static class deleteColumnFamily_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public deleteColumnFamily_argsTupleScheme getScheme() {
-        return new deleteColumnFamily_argsTupleScheme();
+    private static class modifyColumnFamily_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public modifyColumnFamily_argsTupleScheme getScheme() {
+        return new modifyColumnFamily_argsTupleScheme();
       }
     }
 
-    private static class deleteColumnFamily_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<deleteColumnFamily_args> {
+    private static class modifyColumnFamily_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<modifyColumnFamily_args> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, deleteColumnFamily_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, modifyColumnFamily_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         struct.tableName.write(oprot);
-        oprot.writeBinary(struct.column);
+        struct.column.write(oprot);
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, deleteColumnFamily_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, modifyColumnFamily_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         struct.tableName = new TTableName();
         struct.tableName.read(iprot);
         struct.setTableNameIsSet(true);
-        struct.column = iprot.readBinary();
+        struct.column = new TColumnFamilyDescriptor();
+        struct.column.read(iprot);
         struct.setColumnIsSet(true);
       }
     }
@@ -44851,13 +45861,13 @@ public class THBaseService {
     }
   }
 
-  public static class deleteColumnFamily_result implements org.apache.thrift.TBase<deleteColumnFamily_result, deleteColumnFamily_result._Fields>, java.io.Serializable, Cloneable, Comparable<deleteColumnFamily_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("deleteColumnFamily_result");
+  public static class modifyColumnFamily_result implements org.apache.thrift.TBase<modifyColumnFamily_result, modifyColumnFamily_result._Fields>, java.io.Serializable, Cloneable, Comparable<modifyColumnFamily_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("modifyColumnFamily_result");
 
     private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new deleteColumnFamily_resultStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new deleteColumnFamily_resultTupleSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new modifyColumnFamily_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new modifyColumnFamily_resultTupleSchemeFactory();
 
     public @org.apache.thrift.annotation.Nullable TIOError io; // required
 
@@ -44928,13 +45938,13 @@ public class THBaseService {
       tmpMap.put(_Fields.IO, new org.apache.thrift.meta_data.FieldMetaData("io", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TIOError.class)));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(deleteColumnFamily_result.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(modifyColumnFamily_result.class, metaDataMap);
     }
 
-    public deleteColumnFamily_result() {
+    public modifyColumnFamily_result() {
     }
 
-    public deleteColumnFamily_result(
+    public modifyColumnFamily_result(
       TIOError io)
     {
       this();
@@ -44944,14 +45954,14 @@ public class THBaseService {
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public deleteColumnFamily_result(deleteColumnFamily_result other) {
+    public modifyColumnFamily_result(modifyColumnFamily_result other) {
       if (other.isSetIo()) {
         this.io = new TIOError(other.io);
       }
     }
 
-    public deleteColumnFamily_result deepCopy() {
-      return new deleteColumnFamily_result(this);
+    public modifyColumnFamily_result deepCopy() {
+      return new modifyColumnFamily_result(this);
     }
 
     @Override
@@ -44964,7 +45974,7 @@ public class THBaseService {
       return this.io;
     }
 
-    public deleteColumnFamily_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) {
+    public modifyColumnFamily_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) {
       this.io = io;
       return this;
     }
@@ -45024,12 +46034,12 @@ public class THBaseService {
     public boolean equals(java.lang.Object that) {
       if (that == null)
         return false;
-      if (that instanceof deleteColumnFamily_result)
-        return this.equals((deleteColumnFamily_result)that);
+      if (that instanceof modifyColumnFamily_result)
+        return this.equals((modifyColumnFamily_result)that);
       return false;
     }
 
-    public boolean equals(deleteColumnFamily_result that) {
+    public boolean equals(modifyColumnFamily_result that) {
       if (that == null)
         return false;
       if (this == that)
@@ -45059,7 +46069,7 @@ public class THBaseService {
     }
 
     @Override
-    public int compareTo(deleteColumnFamily_result other) {
+    public int compareTo(modifyColumnFamily_result other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -45094,7 +46104,7 @@ public class THBaseService {
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("deleteColumnFamily_result(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("modifyColumnFamily_result(");
       boolean first = true;
 
       sb.append("io:");
@@ -45129,15 +46139,15 @@ public class THBaseService {
       }
     }
 
-    private static class deleteColumnFamily_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public deleteColumnFamily_resultStandardScheme getScheme() {
-        return new deleteColumnFamily_resultStandardScheme();
+    private static class modifyColumnFamily_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public modifyColumnFamily_resultStandardScheme getScheme() {
+        return new modifyColumnFamily_resultStandardScheme();
       }
     }
 
-    private static class deleteColumnFamily_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<deleteColumnFamily_result> {
+    private static class modifyColumnFamily_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<modifyColumnFamily_result> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, deleteColumnFamily_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, modifyColumnFamily_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -45167,7 +46177,7 @@ public class THBaseService {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, deleteColumnFamily_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, modifyColumnFamily_result struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -45182,16 +46192,16 @@ public class THBaseService {
 
     }
 
-    private static class deleteColumnFamily_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public deleteColumnFamily_resultTupleScheme getScheme() {
-        return new deleteColumnFamily_resultTupleScheme();
+    private static class modifyColumnFamily_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public modifyColumnFamily_resultTupleScheme getScheme() {
+        return new modifyColumnFamily_resultTupleScheme();
       }
     }
 
-    private static class deleteColumnFamily_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<deleteColumnFamily_result> {
+    private static class modifyColumnFamily_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<modifyColumnFamily_result> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, deleteColumnFamily_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, modifyColumnFamily_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet optionals = new java.util.BitSet();
         if (struct.isSetIo()) {
@@ -45204,7 +46214,7 @@ public class THBaseService {
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, deleteColumnFamily_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, modifyColumnFamily_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
@@ -45220,34 +46230,25 @@ public class THBaseService {
     }
   }
 
-  public static class modifyColumnFamily_args implements org.apache.thrift.TBase<modifyColumnFamily_args, modifyColumnFamily_args._Fields>, java.io.Serializable, Cloneable, Comparable<modifyColumnFamily_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("modifyColumnFamily_args");
+  public static class modifyTable_args implements org.apache.thrift.TBase<modifyTable_args, modifyTable_args._Fields>, java.io.Serializable, Cloneable, Comparable<modifyTable_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("modifyTable_args");
 
-    private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRUCT, (short)1);
-    private static final org.apache.thrift.protocol.TField COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("column", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+    private static final org.apache.thrift.protocol.TField DESC_FIELD_DESC = new org.apache.thrift.protocol.TField("desc", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new modifyColumnFamily_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new modifyColumnFamily_argsTupleSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new modifyTable_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new modifyTable_argsTupleSchemeFactory();
 
     /**
-     * the tablename to modify column family
-     */
-    public @org.apache.thrift.annotation.Nullable TTableName tableName; // required
-    /**
-     * column family descriptor of column family to be modified
+     * the descriptor of the table to modify
      */
-    public @org.apache.thrift.annotation.Nullable TColumnFamilyDescriptor column; // required
+    public @org.apache.thrift.annotation.Nullable TTableDescriptor desc; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
       /**
-       * the tablename to modify column family
-       */
-      TABLE_NAME((short)1, "tableName"),
-      /**
-       * column family descriptor of column family to be modified
+       * the descriptor of the table to modify
        */
-      COLUMN((short)2, "column");
+      DESC((short)1, "desc");
 
       private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
 
@@ -45263,10 +46264,8 @@ public class THBaseService {
       @org.apache.thrift.annotation.Nullable
       public static _Fields findByThriftId(int fieldId) {
         switch(fieldId) {
-          case 1: // TABLE_NAME
-            return TABLE_NAME;
-          case 2: // COLUMN
-            return COLUMN;
+          case 1: // DESC
+            return DESC;
           default:
             return null;
         }
@@ -45311,125 +46310,78 @@ public class THBaseService {
     public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.REQUIRED, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TTableName.class)));
-      tmpMap.put(_Fields.COLUMN, new org.apache.thrift.meta_data.FieldMetaData("column", org.apache.thrift.TFieldRequirementType.REQUIRED, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TColumnFamilyDescriptor.class)));
+      tmpMap.put(_Fields.DESC, new org.apache.thrift.meta_data.FieldMetaData("desc", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TTableDescriptor.class)));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(modifyColumnFamily_args.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(modifyTable_args.class, metaDataMap);
     }
 
-    public modifyColumnFamily_args() {
+    public modifyTable_args() {
     }
 
-    public modifyColumnFamily_args(
-      TTableName tableName,
-      TColumnFamilyDescriptor column)
+    public modifyTable_args(
+      TTableDescriptor desc)
     {
       this();
-      this.tableName = tableName;
-      this.column = column;
+      this.desc = desc;
     }
 
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public modifyColumnFamily_args(modifyColumnFamily_args other) {
-      if (other.isSetTableName()) {
-        this.tableName = new TTableName(other.tableName);
-      }
-      if (other.isSetColumn()) {
-        this.column = new TColumnFamilyDescriptor(other.column);
+    public modifyTable_args(modifyTable_args other) {
+      if (other.isSetDesc()) {
+        this.desc = new TTableDescriptor(other.desc);
       }
     }
 
-    public modifyColumnFamily_args deepCopy() {
-      return new modifyColumnFamily_args(this);
+    public modifyTable_args deepCopy() {
+      return new modifyTable_args(this);
     }
 
     @Override
     public void clear() {
-      this.tableName = null;
-      this.column = null;
-    }
-
-    /**
-     * the tablename to modify column family
-     */
-    @org.apache.thrift.annotation.Nullable
-    public TTableName getTableName() {
-      return this.tableName;
-    }
-
-    /**
-     * the tablename to modify column family
-     */
-    public modifyColumnFamily_args setTableName(@org.apache.thrift.annotation.Nullable TTableName tableName) {
-      this.tableName = tableName;
-      return this;
-    }
-
-    public void unsetTableName() {
-      this.tableName = null;
-    }
-
-    /** Returns true if field tableName is set (has been assigned a value) and false otherwise */
-    public boolean isSetTableName() {
-      return this.tableName != null;
-    }
-
-    public void setTableNameIsSet(boolean value) {
-      if (!value) {
-        this.tableName = null;
-      }
+      this.desc = null;
     }
 
     /**
-     * column family descriptor of column family to be modified
+     * the descriptor of the table to modify
      */
     @org.apache.thrift.annotation.Nullable
-    public TColumnFamilyDescriptor getColumn() {
-      return this.column;
+    public TTableDescriptor getDesc() {
+      return this.desc;
     }
 
     /**
-     * column family descriptor of column family to be modified
+     * the descriptor of the table to modify
      */
-    public modifyColumnFamily_args setColumn(@org.apache.thrift.annotation.Nullable TColumnFamilyDescriptor column) {
-      this.column = column;
+    public modifyTable_args setDesc(@org.apache.thrift.annotation.Nullable TTableDescriptor desc) {
+      this.desc = desc;
       return this;
     }
 
-    public void unsetColumn() {
-      this.column = null;
+    public void unsetDesc() {
+      this.desc = null;
     }
 
-    /** Returns true if field column is set (has been assigned a value) and false otherwise */
-    public boolean isSetColumn() {
-      return this.column != null;
+    /** Returns true if field desc is set (has been assigned a value) and false otherwise */
+    public boolean isSetDesc() {
+      return this.desc != null;
     }
 
-    public void setColumnIsSet(boolean value) {
+    public void setDescIsSet(boolean value) {
       if (!value) {
-        this.column = null;
+        this.desc = null;
       }
     }
 
     public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
       switch (field) {
-      case TABLE_NAME:
-        if (value == null) {
-          unsetTableName();
-        } else {
-          setTableName((TTableName)value);
-        }
-        break;
-
-      case COLUMN:
+      case DESC:
         if (value == null) {
-          unsetColumn();
+          unsetDesc();
         } else {
-          setColumn((TColumnFamilyDescriptor)value);
+          setDesc((TTableDescriptor)value);
         }
         break;
 
@@ -45439,11 +46391,8 @@ public class THBaseService {
     @org.apache.thrift.annotation.Nullable
     public java.lang.Object getFieldValue(_Fields field) {
       switch (field) {
-      case TABLE_NAME:
-        return getTableName();
-
-      case COLUMN:
-        return getColumn();
+      case DESC:
+        return getDesc();
 
       }
       throw new java.lang.IllegalStateException();
@@ -45456,10 +46405,8 @@ public class THBaseService {
       }
 
       switch (field) {
-      case TABLE_NAME:
-        return isSetTableName();
-      case COLUMN:
-        return isSetColumn();
+      case DESC:
+        return isSetDesc();
       }
       throw new java.lang.IllegalStateException();
     }
@@ -45468,32 +46415,23 @@ public class THBaseService {
     public boolean equals(java.lang.Object that) {
       if (that == null)
         return false;
-      if (that instanceof modifyColumnFamily_args)
-        return this.equals((modifyColumnFamily_args)that);
+      if (that instanceof modifyTable_args)
+        return this.equals((modifyTable_args)that);
       return false;
     }
 
-    public boolean equals(modifyColumnFamily_args that) {
+    public boolean equals(modifyTable_args that) {
       if (that == null)
         return false;
       if (this == that)
         return true;
 
-      boolean this_present_tableName = true && this.isSetTableName();
-      boolean that_present_tableName = true && that.isSetTableName();
-      if (this_present_tableName || that_present_tableName) {
-        if (!(this_present_tableName && that_present_tableName))
-          return false;
-        if (!this.tableName.equals(that.tableName))
-          return false;
-      }
-
-      boolean this_present_column = true && this.isSetColumn();
-      boolean that_present_column = true && that.isSetColumn();
-      if (this_present_column || that_present_column) {
-        if (!(this_present_column && that_present_column))
+      boolean this_present_desc = true && this.isSetDesc();
+      boolean that_present_desc = true && that.isSetDesc();
+      if (this_present_desc || that_present_desc) {
+        if (!(this_present_desc && that_present_desc))
           return false;
-        if (!this.column.equals(that.column))
+        if (!this.desc.equals(that.desc))
           return false;
       }
 
@@ -45504,41 +46442,27 @@ public class THBaseService {
     public int hashCode() {
       int hashCode = 1;
 
-      hashCode = hashCode * 8191 + ((isSetTableName()) ? 131071 : 524287);
-      if (isSetTableName())
-        hashCode = hashCode * 8191 + tableName.hashCode();
-
-      hashCode = hashCode * 8191 + ((isSetColumn()) ? 131071 : 524287);
-      if (isSetColumn())
-        hashCode = hashCode * 8191 + column.hashCode();
+      hashCode = hashCode * 8191 + ((isSetDesc()) ? 131071 : 524287);
+      if (isSetDesc())
+        hashCode = hashCode * 8191 + desc.hashCode();
 
       return hashCode;
     }
 
     @Override
-    public int compareTo(modifyColumnFamily_args other) {
+    public int compareTo(modifyTable_args other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
 
       int lastComparison = 0;
 
-      lastComparison = java.lang.Boolean.valueOf(isSetTableName()).compareTo(other.isSetTableName());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetTableName()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, other.tableName);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = java.lang.Boolean.valueOf(isSetColumn()).compareTo(other.isSetColumn());
+      lastComparison = java.lang.Boolean.valueOf(isSetDesc()).compareTo(other.isSetDesc());
       if (lastComparison != 0) {
         return lastComparison;
       }
-      if (isSetColumn()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.column, other.column);
+      if (isSetDesc()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.desc, other.desc);
         if (lastComparison != 0) {
           return lastComparison;
         }
@@ -45561,22 +46485,14 @@ public class THBaseService {
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("modifyColumnFamily_args(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("modifyTable_args(");
       boolean first = true;
 
-      sb.append("tableName:");
-      if (this.tableName == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.tableName);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("column:");
-      if (this.column == null) {
+      sb.append("desc:");
+      if (this.desc == null) {
         sb.append("null");
       } else {
-        sb.append(this.column);
+        sb.append(this.desc);
       }
       first = false;
       sb.append(")");
@@ -45585,18 +46501,12 @@ public class THBaseService {
 
     public void validate() throws org.apache.thrift.TException {
       // check for required fields
-      if (tableName == null) {
-        throw new org.apache.thrift.protocol.TProtocolException("Required field 'tableName' was not present! Struct: " + toString());
-      }
-      if (column == null) {
-        throw new org.apache.thrift.protocol.TProtocolException("Required field 'column' was not present! Struct: " + toString());
+      if (desc == null) {
+        throw new org.apache.thrift.protocol.TProtocolException("Required field 'desc' was not present! Struct: " + toString());
       }
       // check for sub-struct validity
-      if (tableName != null) {
-        tableName.validate();
-      }
-      if (column != null) {
-        column.validate();
+      if (desc != null) {
+        desc.validate();
       }
     }
 
@@ -45616,15 +46526,15 @@ public class THBaseService {
       }
     }
 
-    private static class modifyColumnFamily_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public modifyColumnFamily_argsStandardScheme getScheme() {
-        return new modifyColumnFamily_argsStandardScheme();
+    private static class modifyTable_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public modifyTable_argsStandardScheme getScheme() {
+        return new modifyTable_argsStandardScheme();
       }
     }
 
-    private static class modifyColumnFamily_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<modifyColumnFamily_args> {
+    private static class modifyTable_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<modifyTable_args> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, modifyColumnFamily_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, modifyTable_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -45634,20 +46544,11 @@ public class THBaseService {
             break;
           }
           switch (schemeField.id) {
-            case 1: // TABLE_NAME
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.tableName = new TTableName();
-                struct.tableName.read(iprot);
-                struct.setTableNameIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 2: // COLUMN
+            case 1: // DESC
               if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.column = new TColumnFamilyDescriptor();
-                struct.column.read(iprot);
-                struct.setColumnIsSet(true);
+                struct.desc = new TTableDescriptor();
+                struct.desc.read(iprot);
+                struct.setDescIsSet(true);
               } else { 
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
@@ -45663,18 +46564,13 @@ public class THBaseService {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, modifyColumnFamily_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, modifyTable_args struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
-        if (struct.tableName != null) {
-          oprot.writeFieldBegin(TABLE_NAME_FIELD_DESC);
-          struct.tableName.write(oprot);
-          oprot.writeFieldEnd();
-        }
-        if (struct.column != null) {
-          oprot.writeFieldBegin(COLUMN_FIELD_DESC);
-          struct.column.write(oprot);
+        if (struct.desc != null) {
+          oprot.writeFieldBegin(DESC_FIELD_DESC);
+          struct.desc.write(oprot);
           oprot.writeFieldEnd();
         }
         oprot.writeFieldStop();
@@ -45683,30 +46579,26 @@ public class THBaseService {
 
     }
 
-    private static class modifyColumnFamily_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public modifyColumnFamily_argsTupleScheme getScheme() {
-        return new modifyColumnFamily_argsTupleScheme();
+    private static class modifyTable_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public modifyTable_argsTupleScheme getScheme() {
+        return new modifyTable_argsTupleScheme();
       }
     }
 
-    private static class modifyColumnFamily_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<modifyColumnFamily_args> {
+    private static class modifyTable_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<modifyTable_args> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, modifyColumnFamily_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, modifyTable_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
-        struct.tableName.write(oprot);
-        struct.column.write(oprot);
+        struct.desc.write(oprot);
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, modifyColumnFamily_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, modifyTable_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
-        struct.tableName = new TTableName();
-        struct.tableName.read(iprot);
-        struct.setTableNameIsSet(true);
-        struct.column = new TColumnFamilyDescriptor();
-        struct.column.read(iprot);
-        struct.setColumnIsSet(true);
+        struct.desc = new TTableDescriptor();
+        struct.desc.read(iprot);
+        struct.setDescIsSet(true);
       }
     }
 
@@ -45715,13 +46607,13 @@ public class THBaseService {
     }
   }
 
-  public static class modifyColumnFamily_result implements org.apache.thrift.TBase<modifyColumnFamily_result, modifyColumnFamily_result._Fields>, java.io.Serializable, Cloneable, Comparable<modifyColumnFamily_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("modifyColumnFamily_result");
+  public static class modifyTable_result implements org.apache.thrift.TBase<modifyTable_result, modifyTable_result._Fields>, java.io.Serializable, Cloneable, Comparable<modifyTable_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("modifyTable_result");
 
     private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new modifyColumnFamily_resultStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new modifyColumnFamily_resultTupleSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new modifyTable_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new modifyTable_resultTupleSchemeFactory();
 
     public @org.apache.thrift.annotation.Nullable TIOError io; // required
 
@@ -45792,13 +46684,13 @@ public class THBaseService {
       tmpMap.put(_Fields.IO, new org.apache.thrift.meta_data.FieldMetaData("io", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TIOError.class)));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(modifyColumnFamily_result.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(modifyTable_result.class, metaDataMap);
     }
 
-    public modifyColumnFamily_result() {
+    public modifyTable_result() {
     }
 
-    public modifyColumnFamily_result(
+    public modifyTable_result(
       TIOError io)
     {
       this();
@@ -45808,14 +46700,14 @@ public class THBaseService {
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public modifyColumnFamily_result(modifyColumnFamily_result other) {
+    public modifyTable_result(modifyTable_result other) {
       if (other.isSetIo()) {
         this.io = new TIOError(other.io);
       }
     }
 
-    public modifyColumnFamily_result deepCopy() {
-      return new modifyColumnFamily_result(this);
+    public modifyTable_result deepCopy() {
+      return new modifyTable_result(this);
     }
 
     @Override
@@ -45828,7 +46720,7 @@ public class THBaseService {
       return this.io;
     }
 
-    public modifyColumnFamily_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) {
+    public modifyTable_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) {
       this.io = io;
       return this;
     }
@@ -45888,12 +46780,12 @@ public class THBaseService {
     public boolean equals(java.lang.Object that) {
       if (that == null)
         return false;
-      if (that instanceof modifyColumnFamily_result)
-        return this.equals((modifyColumnFamily_result)that);
+      if (that instanceof modifyTable_result)
+        return this.equals((modifyTable_result)that);
       return false;
     }
 
-    public boolean equals(modifyColumnFamily_result that) {
+    public boolean equals(modifyTable_result that) {
       if (that == null)
         return false;
       if (this == that)
@@ -45923,7 +46815,7 @@ public class THBaseService {
     }
 
     @Override
-    public int compareTo(modifyColumnFamily_result other) {
+    public int compareTo(modifyTable_result other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -45958,7 +46850,7 @@ public class THBaseService {
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("modifyColumnFamily_result(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("modifyTable_result(");
       boolean first = true;
 
       sb.append("io:");
@@ -45993,15 +46885,15 @@ public class THBaseService {
       }
     }
 
-    private static class modifyColumnFamily_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public modifyColumnFamily_resultStandardScheme getScheme() {
-        return new modifyColumnFamily_resultStandardScheme();
+    private static class modifyTable_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public modifyTable_resultStandardScheme getScheme() {
+        return new modifyTable_resultStandardScheme();
       }
     }
 
-    private static class modifyColumnFamily_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<modifyColumnFamily_result> {
+    private static class modifyTable_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<modifyTable_result> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, modifyColumnFamily_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, modifyTable_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -46031,7 +46923,7 @@ public class THBaseService {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, modifyColumnFamily_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, modifyTable_result struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -46046,16 +46938,16 @@ public class THBaseService {
 
     }
 
-    private static class modifyColumnFamily_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public modifyColumnFamily_resultTupleScheme getScheme() {
-        return new modifyColumnFamily_resultTupleScheme();
+    private static class modifyTable_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public modifyTable_resultTupleScheme getScheme() {
+        return new modifyTable_resultTupleScheme();
       }
     }
 
-    private static class modifyColumnFamily_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<modifyColumnFamily_result> {
+    private static class modifyTable_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<modifyTable_result> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, modifyColumnFamily_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, modifyTable_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet optionals = new java.util.BitSet();
         if (struct.isSetIo()) {
@@ -46068,7 +46960,7 @@ public class THBaseService {
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, modifyColumnFamily_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, modifyTable_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
@@ -46084,25 +46976,25 @@ public class THBaseService {
     }
   }
 
-  public static class modifyTable_args implements org.apache.thrift.TBase<modifyTable_args, modifyTable_args._Fields>, java.io.Serializable, Cloneable, Comparable<modifyTable_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("modifyTable_args");
+  public static class createNamespace_args implements org.apache.thrift.TBase<createNamespace_args, createNamespace_args._Fields>, java.io.Serializable, Cloneable, Comparable<createNamespace_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("createNamespace_args");
 
-    private static final org.apache.thrift.protocol.TField DESC_FIELD_DESC = new org.apache.thrift.protocol.TField("desc", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+    private static final org.apache.thrift.protocol.TField NAMESPACE_DESC_FIELD_DESC = new org.apache.thrift.protocol.TField("namespaceDesc", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new modifyTable_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new modifyTable_argsTupleSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new createNamespace_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new createNamespace_argsTupleSchemeFactory();
 
     /**
-     * the descriptor of the table to modify
+     * descriptor which describes the new namespace
      */
-    public @org.apache.thrift.annotation.Nullable TTableDescriptor desc; // required
+    public @org.apache.thrift.annotation.Nullable TNamespaceDescriptor namespaceDesc; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
       /**
-       * the descriptor of the table to modify
+       * descriptor which describes the new namespace
        */
-      DESC((short)1, "desc");
+      NAMESPACE_DESC((short)1, "namespaceDesc");
 
       private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
 
@@ -46118,8 +47010,8 @@ public class THBaseService {
       @org.apache.thrift.annotation.Nullable
       public static _Fields findByThriftId(int fieldId) {
         switch(fieldId) {
-          case 1: // DESC
-            return DESC;
+          case 1: // NAMESPACE_DESC
+            return NAMESPACE_DESC;
           default:
             return null;
         }
@@ -46164,78 +47056,78 @@ public class THBaseService {
     public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.DESC, new org.apache.thrift.meta_data.FieldMetaData("desc", org.apache.thrift.TFieldRequirementType.REQUIRED, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TTableDescriptor.class)));
+      tmpMap.put(_Fields.NAMESPACE_DESC, new org.apache.thrift.meta_data.FieldMetaData("namespaceDesc", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TNamespaceDescriptor.class)));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(modifyTable_args.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(createNamespace_args.class, metaDataMap);
     }
 
-    public modifyTable_args() {
+    public createNamespace_args() {
     }
 
-    public modifyTable_args(
-      TTableDescriptor desc)
+    public createNamespace_args(
+      TNamespaceDescriptor namespaceDesc)
     {
       this();
-      this.desc = desc;
+      this.namespaceDesc = namespaceDesc;
     }
 
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public modifyTable_args(modifyTable_args other) {
-      if (other.isSetDesc()) {
-        this.desc = new TTableDescriptor(other.desc);
+    public createNamespace_args(createNamespace_args other) {
+      if (other.isSetNamespaceDesc()) {
+        this.namespaceDesc = new TNamespaceDescriptor(other.namespaceDesc);
       }
     }
 
-    public modifyTable_args deepCopy() {
-      return new modifyTable_args(this);
+    public createNamespace_args deepCopy() {
+      return new createNamespace_args(this);
     }
 
     @Override
     public void clear() {
-      this.desc = null;
+      this.namespaceDesc = null;
     }
 
     /**
-     * the descriptor of the table to modify
+     * descriptor which describes the new namespace
      */
     @org.apache.thrift.annotation.Nullable
-    public TTableDescriptor getDesc() {
-      return this.desc;
+    public TNamespaceDescriptor getNamespaceDesc() {
+      return this.namespaceDesc;
     }
 
     /**
-     * the descriptor of the table to modify
+     * descriptor which describes the new namespace
      */
-    public modifyTable_args setDesc(@org.apache.thrift.annotation.Nullable TTableDescriptor desc) {
-      this.desc = desc;
+    public createNamespace_args setNamespaceDesc(@org.apache.thrift.annotation.Nullable TNamespaceDescriptor namespaceDesc) {
+      this.namespaceDesc = namespaceDesc;
       return this;
     }
 
-    public void unsetDesc() {
-      this.desc = null;
+    public void unsetNamespaceDesc() {
+      this.namespaceDesc = null;
     }
 
-    /** Returns true if field desc is set (has been assigned a value) and false otherwise */
-    public boolean isSetDesc() {
-      return this.desc != null;
+    /** Returns true if field namespaceDesc is set (has been assigned a value) and false otherwise */
+    public boolean isSetNamespaceDesc() {
+      return this.namespaceDesc != null;
     }
 
-    public void setDescIsSet(boolean value) {
+    public void setNamespaceDescIsSet(boolean value) {
       if (!value) {
-        this.desc = null;
+        this.namespaceDesc = null;
       }
     }
 
     public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
       switch (field) {
-      case DESC:
+      case NAMESPACE_DESC:
         if (value == null) {
-          unsetDesc();
+          unsetNamespaceDesc();
         } else {
-          setDesc((TTableDescriptor)value);
+          setNamespaceDesc((TNamespaceDescriptor)value);
         }
         break;
 
@@ -46245,8 +47137,8 @@ public class THBaseService {
     @org.apache.thrift.annotation.Nullable
     public java.lang.Object getFieldValue(_Fields field) {
       switch (field) {
-      case DESC:
-        return getDesc();
+      case NAMESPACE_DESC:
+        return getNamespaceDesc();
 
       }
       throw new java.lang.IllegalStateException();
@@ -46259,8 +47151,8 @@ public class THBaseService {
       }
 
       switch (field) {
-      case DESC:
-        return isSetDesc();
+      case NAMESPACE_DESC:
+        return isSetNamespaceDesc();
       }
       throw new java.lang.IllegalStateException();
     }
@@ -46269,23 +47161,23 @@ public class THBaseService {
     public boolean equals(java.lang.Object that) {
       if (that == null)
         return false;
-      if (that instanceof modifyTable_args)
-        return this.equals((modifyTable_args)that);
+      if (that instanceof createNamespace_args)
+        return this.equals((createNamespace_args)that);
       return false;
     }
 
-    public boolean equals(modifyTable_args that) {
+    public boolean equals(createNamespace_args that) {
       if (that == null)
         return false;
       if (this == that)
         return true;
 
-      boolean this_present_desc = true && this.isSetDesc();
-      boolean that_present_desc = true && that.isSetDesc();
-      if (this_present_desc || that_present_desc) {
-        if (!(this_present_desc && that_present_desc))
+      boolean this_present_namespaceDesc = true && this.isSetNamespaceDesc();
+      boolean that_present_namespaceDesc = true && that.isSetNamespaceDesc();
+      if (this_present_namespaceDesc || that_present_namespaceDesc) {
+        if (!(this_present_namespaceDesc && that_present_namespaceDesc))
           return false;
-        if (!this.desc.equals(that.desc))
+        if (!this.namespaceDesc.equals(that.namespaceDesc))
           return false;
       }
 
@@ -46296,27 +47188,27 @@ public class THBaseService {
     public int hashCode() {
       int hashCode = 1;
 
-      hashCode = hashCode * 8191 + ((isSetDesc()) ? 131071 : 524287);
-      if (isSetDesc())
-        hashCode = hashCode * 8191 + desc.hashCode();
+      hashCode = hashCode * 8191 + ((isSetNamespaceDesc()) ? 131071 : 524287);
+      if (isSetNamespaceDesc())
+        hashCode = hashCode * 8191 + namespaceDesc.hashCode();
 
       return hashCode;
     }
 
     @Override
-    public int compareTo(modifyTable_args other) {
+    public int compareTo(createNamespace_args other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
 
       int lastComparison = 0;
 
-      lastComparison = java.lang.Boolean.valueOf(isSetDesc()).compareTo(other.isSetDesc());
+      lastComparison = java.lang.Boolean.valueOf(isSetNamespaceDesc()).compareTo(other.isSetNamespaceDesc());
       if (lastComparison != 0) {
         return lastComparison;
       }
-      if (isSetDesc()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.desc, other.desc);
+      if (isSetNamespaceDesc()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.namespaceDesc, other.namespaceDesc);
         if (lastComparison != 0) {
           return lastComparison;
         }
@@ -46339,14 +47231,14 @@ public class THBaseService {
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("modifyTable_args(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("createNamespace_args(");
       boolean first = true;
 
-      sb.append("desc:");
-      if (this.desc == null) {
+      sb.append("namespaceDesc:");
+      if (this.namespaceDesc == null) {
         sb.append("null");
       } else {
-        sb.append(this.desc);
+        sb.append(this.namespaceDesc);
       }
       first = false;
       sb.append(")");
@@ -46355,12 +47247,12 @@ public class THBaseService {
 
     public void validate() throws org.apache.thrift.TException {
       // check for required fields
-      if (desc == null) {
-        throw new org.apache.thrift.protocol.TProtocolException("Required field 'desc' was not present! Struct: " + toString());
+      if (namespaceDesc == null) {
+        throw new org.apache.thrift.protocol.TProtocolException("Required field 'namespaceDesc' was not present! Struct: " + toString());
       }
       // check for sub-struct validity
-      if (desc != null) {
-        desc.validate();
+      if (namespaceDesc != null) {
+        namespaceDesc.validate();
       }
     }
 
@@ -46380,15 +47272,15 @@ public class THBaseService {
       }
     }
 
-    private static class modifyTable_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public modifyTable_argsStandardScheme getScheme() {
-        return new modifyTable_argsStandardScheme();
+    private static class createNamespace_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public createNamespace_argsStandardScheme getScheme() {
+        return new createNamespace_argsStandardScheme();
       }
     }
 
-    private static class modifyTable_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<modifyTable_args> {
+    private static class createNamespace_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<createNamespace_args> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, modifyTable_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, createNamespace_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -46398,11 +47290,11 @@ public class THBaseService {
             break;
           }
           switch (schemeField.id) {
-            case 1: // DESC
+            case 1: // NAMESPACE_DESC
               if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.desc = new TTableDescriptor();
-                struct.desc.read(iprot);
-                struct.setDescIsSet(true);
+                struct.namespaceDesc = new TNamespaceDescriptor();
+                struct.namespaceDesc.read(iprot);
+                struct.setNamespaceDescIsSet(true);
               } else { 
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
@@ -46418,13 +47310,13 @@ public class THBaseService {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, modifyTable_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, createNamespace_args struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
-        if (struct.desc != null) {
-          oprot.writeFieldBegin(DESC_FIELD_DESC);
-          struct.desc.write(oprot);
+        if (struct.namespaceDesc != null) {
+          oprot.writeFieldBegin(NAMESPACE_DESC_FIELD_DESC);
+          struct.namespaceDesc.write(oprot);
           oprot.writeFieldEnd();
         }
         oprot.writeFieldStop();
@@ -46433,26 +47325,26 @@ public class THBaseService {
 
     }
 
-    private static class modifyTable_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public modifyTable_argsTupleScheme getScheme() {
-        return new modifyTable_argsTupleScheme();
+    private static class createNamespace_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public createNamespace_argsTupleScheme getScheme() {
+        return new createNamespace_argsTupleScheme();
       }
     }
 
-    private static class modifyTable_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<modifyTable_args> {
+    private static class createNamespace_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<createNamespace_args> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, modifyTable_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, createNamespace_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
-        struct.desc.write(oprot);
+        struct.namespaceDesc.write(oprot);
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, modifyTable_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, createNamespace_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
-        struct.desc = new TTableDescriptor();
-        struct.desc.read(iprot);
-        struct.setDescIsSet(true);
+        struct.namespaceDesc = new TNamespaceDescriptor();
+        struct.namespaceDesc.read(iprot);
+        struct.setNamespaceDescIsSet(true);
       }
     }
 
@@ -46461,13 +47353,13 @@ public class THBaseService {
     }
   }
 
-  public static class modifyTable_result implements org.apache.thrift.TBase<modifyTable_result, modifyTable_result._Fields>, java.io.Serializable, Cloneable, Comparable<modifyTable_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("modifyTable_result");
+  public static class createNamespace_result implements org.apache.thrift.TBase<createNamespace_result, createNamespace_result._Fields>, java.io.Serializable, Cloneable, Comparable<createNamespace_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("createNamespace_result");
 
     private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new modifyTable_resultStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new modifyTable_resultTupleSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new createNamespace_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new createNamespace_resultTupleSchemeFactory();
 
     public @org.apache.thrift.annotation.Nullable TIOError io; // required
 
@@ -46538,13 +47430,13 @@ public class THBaseService {
       tmpMap.put(_Fields.IO, new org.apache.thrift.meta_data.FieldMetaData("io", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TIOError.class)));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(modifyTable_result.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(createNamespace_result.class, metaDataMap);
     }
 
-    public modifyTable_result() {
+    public createNamespace_result() {
     }
 
-    public modifyTable_result(
+    public createNamespace_result(
       TIOError io)
     {
       this();
@@ -46554,14 +47446,14 @@ public class THBaseService {
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public modifyTable_result(modifyTable_result other) {
+    public createNamespace_result(createNamespace_result other) {
       if (other.isSetIo()) {
         this.io = new TIOError(other.io);
       }
     }
 
-    public modifyTable_result deepCopy() {
-      return new modifyTable_result(this);
+    public createNamespace_result deepCopy() {
+      return new createNamespace_result(this);
     }
 
     @Override
@@ -46574,7 +47466,7 @@ public class THBaseService {
       return this.io;
     }
 
-    public modifyTable_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) {
+    public createNamespace_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) {
       this.io = io;
       return this;
     }
@@ -46634,12 +47526,12 @@ public class THBaseService {
     public boolean equals(java.lang.Object that) {
       if (that == null)
         return false;
-      if (that instanceof modifyTable_result)
-        return this.equals((modifyTable_result)that);
+      if (that instanceof createNamespace_result)
+        return this.equals((createNamespace_result)that);
       return false;
     }
 
-    public boolean equals(modifyTable_result that) {
+    public boolean equals(createNamespace_result that) {
       if (that == null)
         return false;
       if (this == that)
@@ -46669,7 +47561,7 @@ public class THBaseService {
     }
 
     @Override
-    public int compareTo(modifyTable_result other) {
+    public int compareTo(createNamespace_result other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -46704,7 +47596,7 @@ public class THBaseService {
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("modifyTable_result(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("createNamespace_result(");
       boolean first = true;
 
       sb.append("io:");
@@ -46739,15 +47631,15 @@ public class THBaseService {
       }
     }
 
-    private static class modifyTable_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public modifyTable_resultStandardScheme getScheme() {
-        return new modifyTable_resultStandardScheme();
+    private static class createNamespace_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public createNamespace_resultStandardScheme getScheme() {
+        return new createNamespace_resultStandardScheme();
       }
     }
 
-    private static class modifyTable_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<modifyTable_result> {
+    private static class createNamespace_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<createNamespace_result> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, modifyTable_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, createNamespace_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -46777,7 +47669,7 @@ public class THBaseService {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, modifyTable_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, createNamespace_result struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -46792,16 +47684,16 @@ public class THBaseService {
 
     }
 
-    private static class modifyTable_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public modifyTable_resultTupleScheme getScheme() {
-        return new modifyTable_resultTupleScheme();
+    private static class createNamespace_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public createNamespace_resultTupleScheme getScheme() {
+        return new createNamespace_resultTupleScheme();
       }
     }
 
-    private static class modifyTable_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<modifyTable_result> {
+    private static class createNamespace_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<createNamespace_result> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, modifyTable_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, createNamespace_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet optionals = new java.util.BitSet();
         if (struct.isSetIo()) {
@@ -46814,7 +47706,7 @@ public class THBaseService {
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, modifyTable_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, createNamespace_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
@@ -46830,13 +47722,13 @@ public class THBaseService {
     }
   }
 
-  public static class createNamespace_args implements org.apache.thrift.TBase<createNamespace_args, createNamespace_args._Fields>, java.io.Serializable, Cloneable, Comparable<createNamespace_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("createNamespace_args");
+  public static class modifyNamespace_args implements org.apache.thrift.TBase<modifyNamespace_args, modifyNamespace_args._Fields>, java.io.Serializable, Cloneable, Comparable<modifyNamespace_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("modifyNamespace_args");
 
     private static final org.apache.thrift.protocol.TField NAMESPACE_DESC_FIELD_DESC = new org.apache.thrift.protocol.TField("namespaceDesc", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new createNamespace_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new createNamespace_argsTupleSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new modifyNamespace_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new modifyNamespace_argsTupleSchemeFactory();
 
     /**
      * descriptor which describes the new namespace
@@ -46913,13 +47805,13 @@ public class THBaseService {
       tmpMap.put(_Fields.NAMESPACE_DESC, new org.apache.thrift.meta_data.FieldMetaData("namespaceDesc", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TNamespaceDescriptor.class)));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(createNamespace_args.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(modifyNamespace_args.class, metaDataMap);
     }
 
-    public createNamespace_args() {
+    public modifyNamespace_args() {
     }
 
-    public createNamespace_args(
+    public modifyNamespace_args(
       TNamespaceDescriptor namespaceDesc)
     {
       this();
@@ -46929,14 +47821,14 @@ public class THBaseService {
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public createNamespace_args(createNamespace_args other) {
+    public modifyNamespace_args(modifyNamespace_args other) {
       if (other.isSetNamespaceDesc()) {
         this.namespaceDesc = new TNamespaceDescriptor(other.namespaceDesc);
       }
     }
 
-    public createNamespace_args deepCopy() {
-      return new createNamespace_args(this);
+    public modifyNamespace_args deepCopy() {
+      return new modifyNamespace_args(this);
     }
 
     @Override
@@ -46955,7 +47847,7 @@ public class THBaseService {
     /**
      * descriptor which describes the new namespace
      */
-    public createNamespace_args setNamespaceDesc(@org.apache.thrift.annotation.Nullable TNamespaceDescriptor namespaceDesc) {
+    public modifyNamespace_args setNamespaceDesc(@org.apache.thrift.annotation.Nullable TNamespaceDescriptor namespaceDesc) {
       this.namespaceDesc = namespaceDesc;
       return this;
     }
@@ -47015,12 +47907,12 @@ public class THBaseService {
     public boolean equals(java.lang.Object that) {
       if (that == null)
         return false;
-      if (that instanceof createNamespace_args)
-        return this.equals((createNamespace_args)that);
+      if (that instanceof modifyNamespace_args)
+        return this.equals((modifyNamespace_args)that);
       return false;
     }
 
-    public boolean equals(createNamespace_args that) {
+    public boolean equals(modifyNamespace_args that) {
       if (that == null)
         return false;
       if (this == that)
@@ -47050,7 +47942,7 @@ public class THBaseService {
     }
 
     @Override
-    public int compareTo(createNamespace_args other) {
+    public int compareTo(modifyNamespace_args other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -47085,7 +47977,7 @@ public class THBaseService {
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("createNamespace_args(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("modifyNamespace_args(");
       boolean first = true;
 
       sb.append("namespaceDesc:");
@@ -47126,15 +48018,15 @@ public class THBaseService {
       }
     }
 
-    private static class createNamespace_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public createNamespace_argsStandardScheme getScheme() {
-        return new createNamespace_argsStandardScheme();
+    private static class modifyNamespace_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public modifyNamespace_argsStandardScheme getScheme() {
+        return new modifyNamespace_argsStandardScheme();
       }
     }
 
-    private static class createNamespace_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<createNamespace_args> {
+    private static class modifyNamespace_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<modifyNamespace_args> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, createNamespace_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, modifyNamespace_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -47164,7 +48056,7 @@ public class THBaseService {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, createNamespace_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, modifyNamespace_args struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -47179,22 +48071,22 @@ public class THBaseService {
 
     }
 
-    private static class createNamespace_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public createNamespace_argsTupleScheme getScheme() {
-        return new createNamespace_argsTupleScheme();
+    private static class modifyNamespace_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public modifyNamespace_argsTupleScheme getScheme() {
+        return new modifyNamespace_argsTupleScheme();
       }
     }
 
-    private static class createNamespace_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<createNamespace_args> {
+    private static class modifyNamespace_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<modifyNamespace_args> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, createNamespace_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, modifyNamespace_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         struct.namespaceDesc.write(oprot);
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, createNamespace_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, modifyNamespace_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         struct.namespaceDesc = new TNamespaceDescriptor();
         struct.namespaceDesc.read(iprot);
@@ -47207,13 +48099,13 @@ public class THBaseService {
     }
   }
 
-  public static class createNamespace_result implements org.apache.thrift.TBase<createNamespace_result, createNamespace_result._Fields>, java.io.Serializable, Cloneable, Comparable<createNamespace_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("createNamespace_result");
+  public static class modifyNamespace_result implements org.apache.thrift.TBase<modifyNamespace_result, modifyNamespace_result._Fields>, java.io.Serializable, Cloneable, Comparable<modifyNamespace_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("modifyNamespace_result");
 
     private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new createNamespace_resultStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new createNamespace_resultTupleSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new modifyNamespace_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new modifyNamespace_resultTupleSchemeFactory();
 
     public @org.apache.thrift.annotation.Nullable TIOError io; // required
 
@@ -47284,13 +48176,13 @@ public class THBaseService {
       tmpMap.put(_Fields.IO, new org.apache.thrift.meta_data.FieldMetaData("io", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TIOError.class)));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(createNamespace_result.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(modifyNamespace_result.class, metaDataMap);
     }
 
-    public createNamespace_result() {
+    public modifyNamespace_result() {
     }
 
-    public createNamespace_result(
+    public modifyNamespace_result(
       TIOError io)
     {
       this();
@@ -47300,14 +48192,14 @@ public class THBaseService {
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public createNamespace_result(createNamespace_result other) {
+    public modifyNamespace_result(modifyNamespace_result other) {
       if (other.isSetIo()) {
         this.io = new TIOError(other.io);
       }
     }
 
-    public createNamespace_result deepCopy() {
-      return new createNamespace_result(this);
+    public modifyNamespace_result deepCopy() {
+      return new modifyNamespace_result(this);
     }
 
     @Override
@@ -47320,7 +48212,7 @@ public class THBaseService {
       return this.io;
     }
 
-    public createNamespace_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) {
+    public modifyNamespace_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) {
       this.io = io;
       return this;
     }
@@ -47380,12 +48272,12 @@ public class THBaseService {
     public boolean equals(java.lang.Object that) {
       if (that == null)
         return false;
-      if (that instanceof createNamespace_result)
-        return this.equals((createNamespace_result)that);
+      if (that instanceof modifyNamespace_result)
+        return this.equals((modifyNamespace_result)that);
       return false;
     }
 
-    public boolean equals(createNamespace_result that) {
+    public boolean equals(modifyNamespace_result that) {
       if (that == null)
         return false;
       if (this == that)
@@ -47415,7 +48307,7 @@ public class THBaseService {
     }
 
     @Override
-    public int compareTo(createNamespace_result other) {
+    public int compareTo(modifyNamespace_result other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -47450,7 +48342,7 @@ public class THBaseService {
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("createNamespace_result(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("modifyNamespace_result(");
       boolean first = true;
 
       sb.append("io:");
@@ -47485,15 +48377,15 @@ public class THBaseService {
       }
     }
 
-    private static class createNamespace_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public createNamespace_resultStandardScheme getScheme() {
-        return new createNamespace_resultStandardScheme();
+    private static class modifyNamespace_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public modifyNamespace_resultStandardScheme getScheme() {
+        return new modifyNamespace_resultStandardScheme();
       }
     }
 
-    private static class createNamespace_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<createNamespace_result> {
+    private static class modifyNamespace_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<modifyNamespace_result> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, createNamespace_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, modifyNamespace_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -47523,7 +48415,7 @@ public class THBaseService {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, createNamespace_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, modifyNamespace_result struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -47538,16 +48430,16 @@ public class THBaseService {
 
     }
 
-    private static class createNamespace_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public createNamespace_resultTupleScheme getScheme() {
-        return new createNamespace_resultTupleScheme();
+    private static class modifyNamespace_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public modifyNamespace_resultTupleScheme getScheme() {
+        return new modifyNamespace_resultTupleScheme();
       }
     }
 
-    private static class createNamespace_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<createNamespace_result> {
+    private static class modifyNamespace_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<modifyNamespace_result> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, createNamespace_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, modifyNamespace_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet optionals = new java.util.BitSet();
         if (struct.isSetIo()) {
@@ -47560,7 +48452,7 @@ public class THBaseService {
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, createNamespace_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, modifyNamespace_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
@@ -47576,25 +48468,25 @@ public class THBaseService {
     }
   }
 
-  public static class modifyNamespace_args implements org.apache.thrift.TBase<modifyNamespace_args, modifyNamespace_args._Fields>, java.io.Serializable, Cloneable, Comparable<modifyNamespace_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("modifyNamespace_args");
+  public static class deleteNamespace_args implements org.apache.thrift.TBase<deleteNamespace_args, deleteNamespace_args._Fields>, java.io.Serializable, Cloneable, Comparable<deleteNamespace_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("deleteNamespace_args");
 
-    private static final org.apache.thrift.protocol.TField NAMESPACE_DESC_FIELD_DESC = new org.apache.thrift.protocol.TField("namespaceDesc", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+    private static final org.apache.thrift.protocol.TField NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("name", org.apache.thrift.protocol.TType.STRING, (short)1);
 
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new modifyNamespace_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new modifyNamespace_argsTupleSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new deleteNamespace_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new deleteNamespace_argsTupleSchemeFactory();
 
     /**
-     * descriptor which describes the new namespace
+     * namespace name
      */
-    public @org.apache.thrift.annotation.Nullable TNamespaceDescriptor namespaceDesc; // required
+    public @org.apache.thrift.annotation.Nullable java.lang.String name; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
       /**
-       * descriptor which describes the new namespace
+       * namespace name
        */
-      NAMESPACE_DESC((short)1, "namespaceDesc");
+      NAME((short)1, "name");
 
       private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
 
@@ -47610,8 +48502,8 @@ public class THBaseService {
       @org.apache.thrift.annotation.Nullable
       public static _Fields findByThriftId(int fieldId) {
         switch(fieldId) {
-          case 1: // NAMESPACE_DESC
-            return NAMESPACE_DESC;
+          case 1: // NAME
+            return NAME;
           default:
             return null;
         }
@@ -47656,78 +48548,78 @@ public class THBaseService {
     public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.NAMESPACE_DESC, new org.apache.thrift.meta_data.FieldMetaData("namespaceDesc", org.apache.thrift.TFieldRequirementType.REQUIRED, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TNamespaceDescriptor.class)));
+      tmpMap.put(_Fields.NAME, new org.apache.thrift.meta_data.FieldMetaData("name", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(modifyNamespace_args.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(deleteNamespace_args.class, metaDataMap);
     }
 
-    public modifyNamespace_args() {
+    public deleteNamespace_args() {
     }
 
-    public modifyNamespace_args(
-      TNamespaceDescriptor namespaceDesc)
+    public deleteNamespace_args(
+      java.lang.String name)
     {
       this();
-      this.namespaceDesc = namespaceDesc;
+      this.name = name;
     }
 
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public modifyNamespace_args(modifyNamespace_args other) {
-      if (other.isSetNamespaceDesc()) {
-        this.namespaceDesc = new TNamespaceDescriptor(other.namespaceDesc);
+    public deleteNamespace_args(deleteNamespace_args other) {
+      if (other.isSetName()) {
+        this.name = other.name;
       }
     }
 
-    public modifyNamespace_args deepCopy() {
-      return new modifyNamespace_args(this);
+    public deleteNamespace_args deepCopy() {
+      return new deleteNamespace_args(this);
     }
 
     @Override
     public void clear() {
-      this.namespaceDesc = null;
+      this.name = null;
     }
 
     /**
-     * descriptor which describes the new namespace
+     * namespace name
      */
     @org.apache.thrift.annotation.Nullable
-    public TNamespaceDescriptor getNamespaceDesc() {
-      return this.namespaceDesc;
+    public java.lang.String getName() {
+      return this.name;
     }
 
     /**
-     * descriptor which describes the new namespace
+     * namespace name
      */
-    public modifyNamespace_args setNamespaceDesc(@org.apache.thrift.annotation.Nullable TNamespaceDescriptor namespaceDesc) {
-      this.namespaceDesc = namespaceDesc;
+    public deleteNamespace_args setName(@org.apache.thrift.annotation.Nullable java.lang.String name) {
+      this.name = name;
       return this;
     }
 
-    public void unsetNamespaceDesc() {
-      this.namespaceDesc = null;
+    public void unsetName() {
+      this.name = null;
     }
 
-    /** Returns true if field namespaceDesc is set (has been assigned a value) and false otherwise */
-    public boolean isSetNamespaceDesc() {
-      return this.namespaceDesc != null;
+    /** Returns true if field name is set (has been assigned a value) and false otherwise */
+    public boolean isSetName() {
+      return this.name != null;
     }
 
-    public void setNamespaceDescIsSet(boolean value) {
+    public void setNameIsSet(boolean value) {
       if (!value) {
-        this.namespaceDesc = null;
+        this.name = null;
       }
     }
 
     public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
       switch (field) {
-      case NAMESPACE_DESC:
+      case NAME:
         if (value == null) {
-          unsetNamespaceDesc();
+          unsetName();
         } else {
-          setNamespaceDesc((TNamespaceDescriptor)value);
+          setName((java.lang.String)value);
         }
         break;
 
@@ -47737,8 +48629,8 @@ public class THBaseService {
     @org.apache.thrift.annotation.Nullable
     public java.lang.Object getFieldValue(_Fields field) {
       switch (field) {
-      case NAMESPACE_DESC:
-        return getNamespaceDesc();
+      case NAME:
+        return getName();
 
       }
       throw new java.lang.IllegalStateException();
@@ -47751,8 +48643,8 @@ public class THBaseService {
       }
 
       switch (field) {
-      case NAMESPACE_DESC:
-        return isSetNamespaceDesc();
+      case NAME:
+        return isSetName();
       }
       throw new java.lang.IllegalStateException();
     }
@@ -47761,23 +48653,23 @@ public class THBaseService {
     public boolean equals(java.lang.Object that) {
       if (that == null)
         return false;
-      if (that instanceof modifyNamespace_args)
-        return this.equals((modifyNamespace_args)that);
+      if (that instanceof deleteNamespace_args)
+        return this.equals((deleteNamespace_args)that);
       return false;
     }
 
-    public boolean equals(modifyNamespace_args that) {
+    public boolean equals(deleteNamespace_args that) {
       if (that == null)
         return false;
       if (this == that)
         return true;
 
-      boolean this_present_namespaceDesc = true && this.isSetNamespaceDesc();
-      boolean that_present_namespaceDesc = true && that.isSetNamespaceDesc();
-      if (this_present_namespaceDesc || that_present_namespaceDesc) {
-        if (!(this_present_namespaceDesc && that_present_namespaceDesc))
+      boolean this_present_name = true && this.isSetName();
+      boolean that_present_name = true && that.isSetName();
+      if (this_present_name || that_present_name) {
+        if (!(this_present_name && that_present_name))
           return false;
-        if (!this.namespaceDesc.equals(that.namespaceDesc))
+        if (!this.name.equals(that.name))
           return false;
       }
 
@@ -47788,27 +48680,27 @@ public class THBaseService {
     public int hashCode() {
       int hashCode = 1;
 
-      hashCode = hashCode * 8191 + ((isSetNamespaceDesc()) ? 131071 : 524287);
-      if (isSetNamespaceDesc())
-        hashCode = hashCode * 8191 + namespaceDesc.hashCode();
+      hashCode = hashCode * 8191 + ((isSetName()) ? 131071 : 524287);
+      if (isSetName())
+        hashCode = hashCode * 8191 + name.hashCode();
 
       return hashCode;
     }
 
     @Override
-    public int compareTo(modifyNamespace_args other) {
+    public int compareTo(deleteNamespace_args other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
 
       int lastComparison = 0;
 
-      lastComparison = java.lang.Boolean.valueOf(isSetNamespaceDesc()).compareTo(other.isSetNamespaceDesc());
+      lastComparison = java.lang.Boolean.valueOf(isSetName()).compareTo(other.isSetName());
       if (lastComparison != 0) {
         return lastComparison;
       }
-      if (isSetNamespaceDesc()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.namespaceDesc, other.namespaceDesc);
+      if (isSetName()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.name, other.name);
         if (lastComparison != 0) {
           return lastComparison;
         }
@@ -47831,14 +48723,14 @@ public class THBaseService {
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("modifyNamespace_args(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("deleteNamespace_args(");
       boolean first = true;
 
-      sb.append("namespaceDesc:");
-      if (this.namespaceDesc == null) {
+      sb.append("name:");
+      if (this.name == null) {
         sb.append("null");
       } else {
-        sb.append(this.namespaceDesc);
+        sb.append(this.name);
       }
       first = false;
       sb.append(")");
@@ -47847,13 +48739,10 @@ public class THBaseService {
 
     public void validate() throws org.apache.thrift.TException {
       // check for required fields
-      if (namespaceDesc == null) {
-        throw new org.apache.thrift.protocol.TProtocolException("Required field 'namespaceDesc' was not present! Struct: " + toString());
+      if (name == null) {
+        throw new org.apache.thrift.protocol.TProtocolException("Required field 'name' was not present! Struct: " + toString());
       }
       // check for sub-struct validity
-      if (namespaceDesc != null) {
-        namespaceDesc.validate();
-      }
     }
 
     private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
@@ -47872,15 +48761,15 @@ public class THBaseService {
       }
     }
 
-    private static class modifyNamespace_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public modifyNamespace_argsStandardScheme getScheme() {
-        return new modifyNamespace_argsStandardScheme();
+    private static class deleteNamespace_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public deleteNamespace_argsStandardScheme getScheme() {
+        return new deleteNamespace_argsStandardScheme();
       }
     }
 
-    private static class modifyNamespace_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<modifyNamespace_args> {
+    private static class deleteNamespace_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<deleteNamespace_args> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, modifyNamespace_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, deleteNamespace_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -47890,11 +48779,10 @@ public class THBaseService {
             break;
           }
           switch (schemeField.id) {
-            case 1: // NAMESPACE_DESC
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.namespaceDesc = new TNamespaceDescriptor();
-                struct.namespaceDesc.read(iprot);
-                struct.setNamespaceDescIsSet(true);
+            case 1: // NAME
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.name = iprot.readString();
+                struct.setNameIsSet(true);
               } else { 
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
@@ -47910,13 +48798,13 @@ public class THBaseService {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, modifyNamespace_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, deleteNamespace_args struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
-        if (struct.namespaceDesc != null) {
-          oprot.writeFieldBegin(NAMESPACE_DESC_FIELD_DESC);
-          struct.namespaceDesc.write(oprot);
+        if (struct.name != null) {
+          oprot.writeFieldBegin(NAME_FIELD_DESC);
+          oprot.writeString(struct.name);
           oprot.writeFieldEnd();
         }
         oprot.writeFieldStop();
@@ -47925,26 +48813,25 @@ public class THBaseService {
 
     }
 
-    private static class modifyNamespace_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public modifyNamespace_argsTupleScheme getScheme() {
-        return new modifyNamespace_argsTupleScheme();
+    private static class deleteNamespace_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public deleteNamespace_argsTupleScheme getScheme() {
+        return new deleteNamespace_argsTupleScheme();
       }
     }
 
-    private static class modifyNamespace_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<modifyNamespace_args> {
+    private static class deleteNamespace_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<deleteNamespace_args> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, modifyNamespace_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, deleteNamespace_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
-        struct.namespaceDesc.write(oprot);
+        oprot.writeString(struct.name);
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, modifyNamespace_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, deleteNamespace_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
-        struct.namespaceDesc = new TNamespaceDescriptor();
-        struct.namespaceDesc.read(iprot);
-        struct.setNamespaceDescIsSet(true);
+        struct.name = iprot.readString();
+        struct.setNameIsSet(true);
       }
     }
 
@@ -47953,13 +48840,13 @@ public class THBaseService {
     }
   }
 
-  public static class modifyNamespace_result implements org.apache.thrift.TBase<modifyNamespace_result, modifyNamespace_result._Fields>, java.io.Serializable, Cloneable, Comparable<modifyNamespace_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("modifyNamespace_result");
+  public static class deleteNamespace_result implements org.apache.thrift.TBase<deleteNamespace_result, deleteNamespace_result._Fields>, java.io.Serializable, Cloneable, Comparable<deleteNamespace_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("deleteNamespace_result");
 
     private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new modifyNamespace_resultStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new modifyNamespace_resultTupleSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new deleteNamespace_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new deleteNamespace_resultTupleSchemeFactory();
 
     public @org.apache.thrift.annotation.Nullable TIOError io; // required
 
@@ -48030,13 +48917,13 @@ public class THBaseService {
       tmpMap.put(_Fields.IO, new org.apache.thrift.meta_data.FieldMetaData("io", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TIOError.class)));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(modifyNamespace_result.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(deleteNamespace_result.class, metaDataMap);
     }
 
-    public modifyNamespace_result() {
+    public deleteNamespace_result() {
     }
 
-    public modifyNamespace_result(
+    public deleteNamespace_result(
       TIOError io)
     {
       this();
@@ -48046,14 +48933,14 @@ public class THBaseService {
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public modifyNamespace_result(modifyNamespace_result other) {
+    public deleteNamespace_result(deleteNamespace_result other) {
       if (other.isSetIo()) {
         this.io = new TIOError(other.io);
       }
     }
 
-    public modifyNamespace_result deepCopy() {
-      return new modifyNamespace_result(this);
+    public deleteNamespace_result deepCopy() {
+      return new deleteNamespace_result(this);
     }
 
     @Override
@@ -48066,7 +48953,7 @@ public class THBaseService {
       return this.io;
     }
 
-    public modifyNamespace_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) {
+    public deleteNamespace_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) {
       this.io = io;
       return this;
     }
@@ -48126,12 +49013,12 @@ public class THBaseService {
     public boolean equals(java.lang.Object that) {
       if (that == null)
         return false;
-      if (that instanceof modifyNamespace_result)
-        return this.equals((modifyNamespace_result)that);
+      if (that instanceof deleteNamespace_result)
+        return this.equals((deleteNamespace_result)that);
       return false;
     }
 
-    public boolean equals(modifyNamespace_result that) {
+    public boolean equals(deleteNamespace_result that) {
       if (that == null)
         return false;
       if (this == that)
@@ -48161,7 +49048,7 @@ public class THBaseService {
     }
 
     @Override
-    public int compareTo(modifyNamespace_result other) {
+    public int compareTo(deleteNamespace_result other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -48196,7 +49083,7 @@ public class THBaseService {
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("modifyNamespace_result(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("deleteNamespace_result(");
       boolean first = true;
 
       sb.append("io:");
@@ -48231,15 +49118,15 @@ public class THBaseService {
       }
     }
 
-    private static class modifyNamespace_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public modifyNamespace_resultStandardScheme getScheme() {
-        return new modifyNamespace_resultStandardScheme();
+    private static class deleteNamespace_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public deleteNamespace_resultStandardScheme getScheme() {
+        return new deleteNamespace_resultStandardScheme();
       }
     }
 
-    private static class modifyNamespace_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<modifyNamespace_result> {
+    private static class deleteNamespace_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<deleteNamespace_result> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, modifyNamespace_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, deleteNamespace_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -48269,7 +49156,7 @@ public class THBaseService {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, modifyNamespace_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, deleteNamespace_result struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -48284,16 +49171,16 @@ public class THBaseService {
 
     }
 
-    private static class modifyNamespace_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public modifyNamespace_resultTupleScheme getScheme() {
-        return new modifyNamespace_resultTupleScheme();
+    private static class deleteNamespace_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public deleteNamespace_resultTupleScheme getScheme() {
+        return new deleteNamespace_resultTupleScheme();
       }
     }
 
-    private static class modifyNamespace_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<modifyNamespace_result> {
+    private static class deleteNamespace_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<deleteNamespace_result> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, modifyNamespace_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, deleteNamespace_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet optionals = new java.util.BitSet();
         if (struct.isSetIo()) {
@@ -48306,7 +49193,7 @@ public class THBaseService {
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, modifyNamespace_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, deleteNamespace_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
@@ -48322,23 +49209,23 @@ public class THBaseService {
     }
   }
 
-  public static class deleteNamespace_args implements org.apache.thrift.TBase<deleteNamespace_args, deleteNamespace_args._Fields>, java.io.Serializable, Cloneable, Comparable<deleteNamespace_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("deleteNamespace_args");
+  public static class getNamespaceDescriptor_args implements org.apache.thrift.TBase<getNamespaceDescriptor_args, getNamespaceDescriptor_args._Fields>, java.io.Serializable, Cloneable, Comparable<getNamespaceDescriptor_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getNamespaceDescriptor_args");
 
     private static final org.apache.thrift.protocol.TField NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("name", org.apache.thrift.protocol.TType.STRING, (short)1);
 
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new deleteNamespace_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new deleteNamespace_argsTupleSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getNamespaceDescriptor_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getNamespaceDescriptor_argsTupleSchemeFactory();
 
     /**
-     * namespace name
+     * name of namespace descriptor
      */
     public @org.apache.thrift.annotation.Nullable java.lang.String name; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
       /**
-       * namespace name
+       * name of namespace descriptor
        */
       NAME((short)1, "name");
 
@@ -48405,13 +49292,13 @@ public class THBaseService {
       tmpMap.put(_Fields.NAME, new org.apache.thrift.meta_data.FieldMetaData("name", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(deleteNamespace_args.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getNamespaceDescriptor_args.class, metaDataMap);
     }
 
-    public deleteNamespace_args() {
+    public getNamespaceDescriptor_args() {
     }
 
-    public deleteNamespace_args(
+    public getNamespaceDescriptor_args(
       java.lang.String name)
     {
       this();
@@ -48421,14 +49308,14 @@ public class THBaseService {
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public deleteNamespace_args(deleteNamespace_args other) {
+    public getNamespaceDescriptor_args(getNamespaceDescriptor_args other) {
       if (other.isSetName()) {
         this.name = other.name;
       }
     }
 
-    public deleteNamespace_args deepCopy() {
-      return new deleteNamespace_args(this);
+    public getNamespaceDescriptor_args deepCopy() {
+      return new getNamespaceDescriptor_args(this);
     }
 
     @Override
@@ -48437,7 +49324,7 @@ public class THBaseService {
     }
 
     /**
-     * namespace name
+     * name of namespace descriptor
      */
     @org.apache.thrift.annotation.Nullable
     public java.lang.String getName() {
@@ -48445,9 +49332,9 @@ public class THBaseService {
     }
 
     /**
-     * namespace name
+     * name of namespace descriptor
      */
-    public deleteNamespace_args setName(@org.apache.thrift.annotation.Nullable java.lang.String name) {
+    public getNamespaceDescriptor_args setName(@org.apache.thrift.annotation.Nullable java.lang.String name) {
       this.name = name;
       return this;
     }
@@ -48507,12 +49394,12 @@ public class THBaseService {
     public boolean equals(java.lang.Object that) {
       if (that == null)
         return false;
-      if (that instanceof deleteNamespace_args)
-        return this.equals((deleteNamespace_args)that);
+      if (that instanceof getNamespaceDescriptor_args)
+        return this.equals((getNamespaceDescriptor_args)that);
       return false;
     }
 
-    public boolean equals(deleteNamespace_args that) {
+    public boolean equals(getNamespaceDescriptor_args that) {
       if (that == null)
         return false;
       if (this == that)
@@ -48542,7 +49429,7 @@ public class THBaseService {
     }
 
     @Override
-    public int compareTo(deleteNamespace_args other) {
+    public int compareTo(getNamespaceDescriptor_args other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -48577,7 +49464,7 @@ public class THBaseService {
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("deleteNamespace_args(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("getNamespaceDescriptor_args(");
       boolean first = true;
 
       sb.append("name:");
@@ -48615,15 +49502,15 @@ public class THBaseService {
       }
     }
 
-    private static class deleteNamespace_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public deleteNamespace_argsStandardScheme getScheme() {
-        return new deleteNamespace_argsStandardScheme();
+    private static class getNamespaceDescriptor_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getNamespaceDescriptor_argsStandardScheme getScheme() {
+        return new getNamespaceDescriptor_argsStandardScheme();
       }
     }
 
-    private static class deleteNamespace_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<deleteNamespace_args> {
+    private static class getNamespaceDescriptor_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<getNamespaceDescriptor_args> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, deleteNamespace_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, getNamespaceDescriptor_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -48652,7 +49539,7 @@ public class THBaseService {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, deleteNamespace_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, getNamespaceDescriptor_args struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -48667,22 +49554,22 @@ public class THBaseService {
 
     }
 
-    private static class deleteNamespace_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public deleteNamespace_argsTupleScheme getScheme() {
-        return new deleteNamespace_argsTupleScheme();
+    private static class getNamespaceDescriptor_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getNamespaceDescriptor_argsTupleScheme getScheme() {
+        return new getNamespaceDescriptor_argsTupleScheme();
       }
     }
 
-    private static class deleteNamespace_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<deleteNamespace_args> {
+    private static class getNamespaceDescriptor_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<getNamespaceDescriptor_args> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, deleteNamespace_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, getNamespaceDescriptor_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         oprot.writeString(struct.name);
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, deleteNamespace_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, getNamespaceDescriptor_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         struct.name = iprot.readString();
         struct.setNameIsSet(true);
@@ -48694,18 +49581,21 @@ public class THBaseService {
     }
   }
 
-  public static class deleteNamespace_result implements org.apache.thrift.TBase<deleteNamespace_result, deleteNamespace_result._Fields>, java.io.Serializable, Cloneable, Comparable<deleteNamespace_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("deleteNamespace_result");
+  public static class getNamespaceDescriptor_result implements org.apache.thrift.TBase<getNamespaceDescriptor_result, getNamespaceDescriptor_result._Fields>, java.io.Serializable, Cloneable, Comparable<getNamespaceDescriptor_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getNamespaceDescriptor_result");
 
+    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0);
     private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new deleteNamespace_resultStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new deleteNamespace_resultTupleSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getNamespaceDescriptor_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getNamespaceDescriptor_resultTupleSchemeFactory();
 
+    public @org.apache.thrift.annotation.Nullable TNamespaceDescriptor success; // required
     public @org.apache.thrift.annotation.Nullable TIOError io; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      SUCCESS((short)0, "success"),
       IO((short)1, "io");
 
       private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
@@ -48722,6 +49612,8 @@ public class THBaseService {
       @org.apache.thrift.annotation.Nullable
       public static _Fields findByThriftId(int fieldId) {
         switch(fieldId) {
+          case 0: // SUCCESS
+            return SUCCESS;
           case 1: // IO
             return IO;
           default:
@@ -48768,46 +49660,79 @@ public class THBaseService {
     public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TNamespaceDescriptor.class)));
       tmpMap.put(_Fields.IO, new org.apache.thrift.meta_data.FieldMetaData("io", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TIOError.class)));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(deleteNamespace_result.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getNamespaceDescriptor_result.class, metaDataMap);
     }
 
-    public deleteNamespace_result() {
+    public getNamespaceDescriptor_result() {
     }
 
-    public deleteNamespace_result(
+    public getNamespaceDescriptor_result(
+      TNamespaceDescriptor success,
       TIOError io)
     {
       this();
+      this.success = success;
       this.io = io;
     }
 
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public deleteNamespace_result(deleteNamespace_result other) {
+    public getNamespaceDescriptor_result(getNamespaceDescriptor_result other) {
+      if (other.isSetSuccess()) {
+        this.success = new TNamespaceDescriptor(other.success);
+      }
       if (other.isSetIo()) {
         this.io = new TIOError(other.io);
       }
     }
 
-    public deleteNamespace_result deepCopy() {
-      return new deleteNamespace_result(this);
+    public getNamespaceDescriptor_result deepCopy() {
+      return new getNamespaceDescriptor_result(this);
     }
 
     @Override
     public void clear() {
+      this.success = null;
       this.io = null;
     }
 
     @org.apache.thrift.annotation.Nullable
+    public TNamespaceDescriptor getSuccess() {
+      return this.success;
+    }
+
+    public getNamespaceDescriptor_result setSuccess(@org.apache.thrift.annotation.Nullable TNamespaceDescriptor success) {
+      this.success = success;
+      return this;
+    }
+
+    public void unsetSuccess() {
+      this.success = null;
+    }
+
+    /** Returns true if field success is set (has been assigned a value) and false otherwise */
+    public boolean isSetSuccess() {
+      return this.success != null;
+    }
+
+    public void setSuccessIsSet(boolean value) {
+      if (!value) {
+        this.success = null;
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
     public TIOError getIo() {
       return this.io;
     }
 
-    public deleteNamespace_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) {
+    public getNamespaceDescriptor_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) {
       this.io = io;
       return this;
     }
@@ -48829,6 +49754,14 @@ public class THBaseService {
 
     public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
       switch (field) {
+      case SUCCESS:
+        if (value == null) {
... 2522 lines suppressed ...


[hbase] 03/09: HBASE-23304: RPCs needed for client meta information lookup (#904)

Posted by nd...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

ndimiduk pushed a commit to branch HBASE-18095/client-locate-meta-no-zookeeper
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit f44f7e552209240d091c3f64fccfb35d2e5ecbf2
Author: Bharath Vissapragada <bh...@apache.org>
AuthorDate: Thu Dec 19 11:29:25 2019 -0800

    HBASE-23304: RPCs needed for client meta information lookup (#904)
    
    * HBASE-23304: RPCs needed for client meta information lookup
    
    This patch implements the RPCs needed for the meta information
    lookup during connection init. New tests added to cover the RPC
    code paths. HBASE-23305 builds on this to implement the client
    side logic.
    
    Fixed a bunch of checkstyle nits around the places the patch
    touches.
    
    Signed-off-by: Andrew Purtell <ap...@apache.org>
---
 .../hadoop/hbase/shaded/protobuf/ProtobufUtil.java |   4 +-
 .../src/main/protobuf/Master.proto                 |  44 ++++++
 .../hadoop/hbase/master/MasterRpcServices.java     |  85 ++++++++---
 .../hbase/master/TestClientMetaServiceRPCs.java    | 164 +++++++++++++++++++++
 4 files changed, 275 insertions(+), 22 deletions(-)

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 2adcea9..23f5c08 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
@@ -376,7 +376,9 @@ public final class ProtobufUtil {
    * @see #toServerName(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName)
    */
   public static HBaseProtos.ServerName toServerName(final ServerName serverName) {
-    if (serverName == null) return null;
+    if (serverName == null) {
+      return null;
+    }
     HBaseProtos.ServerName.Builder builder =
       HBaseProtos.ServerName.newBuilder();
     builder.setHostName(serverName.getHostname());
diff --git a/hbase-protocol-shaded/src/main/protobuf/Master.proto b/hbase-protocol-shaded/src/main/protobuf/Master.proto
index 69377a6..e88ddc4 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Master.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Master.proto
@@ -1200,3 +1200,47 @@ service HbckService {
   rpc FixMeta(FixMetaRequest)
     returns(FixMetaResponse);
 }
+
+/** Request and response to get the clusterID for this cluster */
+message GetClusterIdRequest {
+}
+message GetClusterIdResponse {
+  /** Not set if cluster ID could not be determined. */
+  optional string cluster_id = 1;
+}
+
+/** Request and response to get the currently active master name for this cluster */
+message GetActiveMasterRequest {
+}
+message GetActiveMasterResponse {
+  /** Not set if an active master could not be determined. */
+  optional ServerName server_name = 1;
+}
+
+/** Request and response to get the current list of meta region locations */
+message GetMetaRegionLocationsRequest {
+}
+message GetMetaRegionLocationsResponse {
+  /** Not set if meta region locations could not be determined. */
+  repeated RegionLocation meta_locations = 1;
+}
+
+/**
+ * Implements all the RPCs needed by clients to look up cluster meta information needed for connection establishment.
+ */
+service ClientMetaService {
+  /**
+   * Get Cluster ID for this cluster.
+   */
+  rpc GetClusterId(GetClusterIdRequest) returns(GetClusterIdResponse);
+
+  /**
+   * Get active master server name for this cluster.
+   */
+  rpc GetActiveMaster(GetActiveMasterRequest) returns(GetActiveMasterResponse);
+
+  /**
+   * Get current meta replicas' region locations.
+   */
+  rpc GetMetaRegionLocations(GetMetaRegionLocationsRequest) returns(GetMetaRegionLocationsResponse);
+}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index 5fb5f15..47259dc 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.master;
 
 import static org.apache.hadoop.hbase.master.MasterWalManager.META_FILTER;
-
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.net.BindException;
@@ -30,6 +29,7 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Optional;
 import java.util.Set;
 import java.util.stream.Collectors;
 import org.apache.hadoop.conf.Configuration;
@@ -37,6 +37,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.ClusterMetricsBuilder;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.Server;
@@ -116,11 +117,9 @@ import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-
 import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
 import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
 import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
-
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos;
@@ -161,6 +160,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceReq
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ClearDeadServersRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ClearDeadServersResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ClientMetaService;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest;
@@ -185,12 +185,18 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProced
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.FixMetaRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.FixMetaResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetActiveMasterRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetActiveMasterResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterIdRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterIdResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetLocksRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetLocksResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetMetaRegionLocationsRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetMetaRegionLocationsResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest;
@@ -351,9 +357,10 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.Snapshot
  */
 @InterfaceAudience.Private
 @SuppressWarnings("deprecation")
-public class MasterRpcServices extends RSRpcServices
-      implements MasterService.BlockingInterface, RegionServerStatusService.BlockingInterface,
-        LockService.BlockingInterface, HbckService.BlockingInterface {
+public class MasterRpcServices extends RSRpcServices implements
+    MasterService.BlockingInterface, RegionServerStatusService.BlockingInterface,
+    LockService.BlockingInterface, HbckService.BlockingInterface,
+    ClientMetaService.BlockingInterface {
   private static final Logger LOG = LoggerFactory.getLogger(MasterRpcServices.class.getName());
   private static final Logger AUDITLOG =
       LoggerFactory.getLogger("SecurityLogger."+MasterRpcServices.class.getName());
@@ -362,7 +369,7 @@ public class MasterRpcServices extends RSRpcServices
 
   /**
    * @return Subset of configuration to pass initializing regionservers: e.g.
-   * the filesystem to use and root directory to use.
+   *     the filesystem to use and root directory to use.
    */
   private RegionServerStartupResponse.Builder createConfigurationSubset() {
     RegionServerStartupResponse.Builder resp = addConfig(
@@ -488,15 +495,17 @@ public class MasterRpcServices extends RSRpcServices
   protected List<BlockingServiceAndInterface> getServices() {
     List<BlockingServiceAndInterface> bssi = new ArrayList<>(5);
     bssi.add(new BlockingServiceAndInterface(
-      MasterService.newReflectiveBlockingService(this),
-      MasterService.BlockingInterface.class));
+        MasterService.newReflectiveBlockingService(this),
+        MasterService.BlockingInterface.class));
     bssi.add(new BlockingServiceAndInterface(
-      RegionServerStatusService.newReflectiveBlockingService(this),
-      RegionServerStatusService.BlockingInterface.class));
+        RegionServerStatusService.newReflectiveBlockingService(this),
+        RegionServerStatusService.BlockingInterface.class));
     bssi.add(new BlockingServiceAndInterface(LockService.newReflectiveBlockingService(this),
         LockService.BlockingInterface.class));
     bssi.add(new BlockingServiceAndInterface(HbckService.newReflectiveBlockingService(this),
         HbckService.BlockingInterface.class));
+    bssi.add(new BlockingServiceAndInterface(ClientMetaService.newReflectiveBlockingService(this),
+        ClientMetaService.BlockingInterface.class));
     bssi.addAll(super.getServices());
     return bssi;
   }
@@ -623,7 +632,9 @@ public class MasterRpcServices extends RSRpcServices
 
       final byte[] regionName = req.getRegion().getValue().toByteArray();
       final RegionInfo regionInfo = master.getAssignmentManager().getRegionInfo(regionName);
-      if (regionInfo == null) throw new UnknownRegionException(Bytes.toStringBinary(regionName));
+      if (regionInfo == null) {
+        throw new UnknownRegionException(Bytes.toStringBinary(regionName));
+      }
 
       final AssignRegionResponse arr = AssignRegionResponse.newBuilder().build();
       if (master.cpHost != null) {
@@ -668,7 +679,7 @@ public class MasterRpcServices extends RSRpcServices
 
   @Override
   public CreateTableResponse createTable(RpcController controller, CreateTableRequest req)
-  throws ServiceException {
+      throws ServiceException {
     TableDescriptor tableDescriptor = ProtobufUtil.toTableDescriptor(req.getTableSchema());
     byte [][] splitKeys = ProtobufUtil.getSplitKeysArray(req);
     try {
@@ -1065,7 +1076,7 @@ public class MasterRpcServices extends RSRpcServices
    * Get list of TableDescriptors for requested tables.
    * @param c Unused (set to null).
    * @param req GetTableDescriptorsRequest that contains:
-   * - tableNames: requested tables, or if empty, all are requested
+   *     - tableNames: requested tables, or if empty, all are requested.
    * @return GetTableDescriptorsResponse
    * @throws ServiceException
    */
@@ -1209,9 +1220,9 @@ public class MasterRpcServices extends RSRpcServices
   /**
    * Checks if the specified snapshot is done.
    * @return true if the snapshot is in file system ready to use,
-   *   false if the snapshot is in the process of completing
+   *     false if the snapshot is in the process of completing
    * @throws ServiceException wrapping UnknownSnapshotException if invalid snapshot, or
-   *  a wrapped HBaseSnapshotException with progress failure reason.
+   *     a wrapped HBaseSnapshotException with progress failure reason.
    */
   @Override
   public IsSnapshotDoneResponse isSnapshotDone(RpcController controller,
@@ -1453,7 +1464,9 @@ public class MasterRpcServices extends RSRpcServices
 
       final byte[] regionName = request.getRegion().getValue().toByteArray();
       final RegionInfo hri = master.getAssignmentManager().getRegionInfo(regionName);
-      if (hri == null) throw new UnknownRegionException(Bytes.toStringBinary(regionName));
+      if (hri == null) {
+        throw new UnknownRegionException(Bytes.toStringBinary(regionName));
+      }
 
       if (master.cpHost != null) {
         master.cpHost.preRegionOffline(hri);
@@ -2314,8 +2327,8 @@ public class MasterRpcServices extends RSRpcServices
             report.getRegionSize(), now);
         }
       } else {
-        LOG.debug(
-          "Received region space usage report but HMaster is not ready to process it, skipping");
+        LOG.debug("Received region space usage report but HMaster is not ready to process it, "
+            + "skipping");
       }
       return RegionSpaceUseReportResponse.newBuilder().build();
     } catch (Exception e) {
@@ -2351,8 +2364,8 @@ public class MasterRpcServices extends RSRpcServices
         }
         return builder.build();
       } else {
-        LOG.debug(
-          "Received space quota region size report but HMaster is not ready to process it, skipping");
+        LOG.debug("Received space quota region size report but HMaster is not ready to process it,"
+            + "skipping");
       }
       return builder.build();
     } catch (Exception e) {
@@ -2896,4 +2909,34 @@ public class MasterRpcServices extends RSRpcServices
     return true;
   }
 
+  @Override
+  public GetClusterIdResponse getClusterId(RpcController rpcController, GetClusterIdRequest request)
+      throws ServiceException {
+    GetClusterIdResponse.Builder resp = GetClusterIdResponse.newBuilder();
+    String clusterId = master.getClusterId();
+    if (clusterId != null) {
+      resp.setClusterId(clusterId);
+    }
+    return resp.build();
+  }
+
+  @Override
+  public GetActiveMasterResponse getActiveMaster(RpcController rpcController,
+      GetActiveMasterRequest request) throws ServiceException {
+    GetActiveMasterResponse.Builder resp = GetActiveMasterResponse.newBuilder();
+    Optional<ServerName> serverName = master.getActiveMaster();
+    serverName.ifPresent(name -> resp.setServerName(ProtobufUtil.toServerName(name)));
+    return resp.build();
+  }
+
+  @Override
+  public GetMetaRegionLocationsResponse getMetaRegionLocations(RpcController rpcController,
+      GetMetaRegionLocationsRequest request) throws ServiceException {
+    GetMetaRegionLocationsResponse.Builder response = GetMetaRegionLocationsResponse.newBuilder();
+    Optional<List<HRegionLocation>> metaLocations =
+        master.getMetaRegionLocationCache().getMetaRegionLocations();
+    metaLocations.ifPresent(hRegionLocations -> hRegionLocations.forEach(
+      location -> response.addMetaLocations(ProtobufUtil.toRegionLocation(location))));
+    return response.build();
+  }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClientMetaServiceRPCs.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClientMetaServiceRPCs.java
new file mode 100644
index 0000000..428aee2
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClientMetaServiceRPCs.java
@@ -0,0 +1,164 @@
+/*
+ *
+ * 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.master;
+
+import static org.apache.hadoop.hbase.HConstants.DEFAULT_HBASE_RPC_TIMEOUT;
+import static org.apache.hadoop.hbase.HConstants.HBASE_RPC_TIMEOUT_KEY;
+import static org.junit.Assert.assertEquals;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+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.ServerName;
+import org.apache.hadoop.hbase.StartMiniClusterOption;
+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.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.JVMClusterUtil;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ClientMetaService;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetActiveMasterRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetActiveMasterResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterIdRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterIdResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetMetaRegionLocationsRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetMetaRegionLocationsResponse;
+
+@Category({MediumTests.class, MasterTests.class})
+public class TestClientMetaServiceRPCs {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestClientMetaServiceRPCs.class);
+
+  // Total number of masters (active + stand by) for the purpose of this test.
+  private static final int MASTER_COUNT = 3;
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static Configuration conf;
+  private static int rpcTimeout;
+  private static RpcClient rpcClient;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    // Start the mini cluster with stand-by masters.
+    StartMiniClusterOption.Builder builder = StartMiniClusterOption.builder();
+    builder.numMasters(MASTER_COUNT).numRegionServers(3);
+    TEST_UTIL.startMiniCluster(builder.build());
+    conf = TEST_UTIL.getConfiguration();
+    rpcTimeout = (int) Math.min(Integer.MAX_VALUE, TimeUnit.MILLISECONDS.toNanos(
+        conf.getLong(HBASE_RPC_TIMEOUT_KEY, DEFAULT_HBASE_RPC_TIMEOUT)));
+    rpcClient = RpcClientFactory.createClient(conf,
+        TEST_UTIL.getMiniHBaseCluster().getMaster().getClusterId());
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    if (rpcClient != null) {
+      rpcClient.close();
+    }
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  private static ClientMetaService.BlockingInterface getMasterStub(ServerName server)
+      throws IOException {
+    return ClientMetaService.newBlockingStub(
+        rpcClient.createBlockingRpcChannel(server, User.getCurrent(), rpcTimeout));
+  }
+
+  private static HBaseRpcController getRpcController() {
+    return RpcControllerFactory.instantiate(conf).newController();
+  }
+
+  /**
+   * Verifies the cluster ID from all running masters.
+   */
+  @Test public void TestClusterID() throws Exception {
+    HBaseRpcController rpcController = getRpcController();
+    String clusterID = TEST_UTIL.getMiniHBaseCluster().getMaster().getClusterId();
+    int rpcCount = 0;
+    for (JVMClusterUtil.MasterThread masterThread:
+        TEST_UTIL.getMiniHBaseCluster().getMasterThreads()) {
+      ClientMetaService.BlockingInterface stub =
+          getMasterStub(masterThread.getMaster().getServerName());
+      GetClusterIdResponse resp =
+          stub.getClusterId(rpcController, GetClusterIdRequest.getDefaultInstance());
+      assertEquals(clusterID, resp.getClusterId());
+      rpcCount++;
+    }
+    assertEquals(MASTER_COUNT, rpcCount);
+  }
+
+  /**
+   * Verifies the active master ServerName as seen by all masters.
+   */
+  @Test public void TestActiveMaster() throws Exception {
+    HBaseRpcController rpcController = getRpcController();
+    ServerName activeMaster = TEST_UTIL.getMiniHBaseCluster().getMaster().getServerName();
+    int rpcCount = 0;
+    for (JVMClusterUtil.MasterThread masterThread:
+        TEST_UTIL.getMiniHBaseCluster().getMasterThreads()) {
+      ClientMetaService.BlockingInterface stub =
+          getMasterStub(masterThread.getMaster().getServerName());
+      GetActiveMasterResponse resp =
+          stub.getActiveMaster(rpcController, GetActiveMasterRequest.getDefaultInstance());
+      assertEquals(activeMaster, ProtobufUtil.toServerName(resp.getServerName()));
+      rpcCount++;
+    }
+    assertEquals(MASTER_COUNT, rpcCount);
+  }
+
+  /**
+   * Verifies that the meta region locations RPC returns consistent results across all masters.
+   */
+  @Test public void TestMetaLocations() throws Exception {
+    HBaseRpcController rpcController = getRpcController();
+    List<HRegionLocation> metaLocations = TEST_UTIL.getMiniHBaseCluster().getMaster()
+        .getMetaRegionLocationCache().getMetaRegionLocations().get();
+    Collections.sort(metaLocations);
+    int rpcCount = 0;
+    for (JVMClusterUtil.MasterThread masterThread:
+      TEST_UTIL.getMiniHBaseCluster().getMasterThreads()) {
+      ClientMetaService.BlockingInterface stub =
+          getMasterStub(masterThread.getMaster().getServerName());
+      GetMetaRegionLocationsResponse resp = stub.getMetaRegionLocations(
+          rpcController, GetMetaRegionLocationsRequest.getDefaultInstance());
+      List<HRegionLocation> result = new ArrayList<>();
+      resp.getMetaLocationsList().forEach(
+        location -> result.add(ProtobufUtil.toRegionLocation(location)));
+      Collections.sort(result);
+      assertEquals(metaLocations, result);
+      rpcCount++;
+    }
+    assertEquals(MASTER_COUNT, rpcCount);
+  }
+}


[hbase] 02/09: HBASE-23281: Track meta region locations in masters (#830)

Posted by nd...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

ndimiduk pushed a commit to branch HBASE-18095/client-locate-meta-no-zookeeper
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit f3aaba1bbd15bb382f977707a3a19f97101d0a0b
Author: Bharath Vissapragada <bh...@apache.org>
AuthorDate: Wed Dec 4 15:26:58 2019 -0800

    HBASE-23281: Track meta region locations in masters (#830)
    
    * HBASE-23281: Track meta region changes on masters
    
    This patch adds a simple cache that tracks the meta region replica
    locations. It keeps an eye on the region movements so that the
    cached locations are not stale.
    
    This information is used for servicing client RPCs for connections
    that use master based registry (HBASE-18095). The RPC end points
    will be added in a separate patch.
    
    Signed-off-by: Nick Dimiduk <nd...@apache.org>
---
 .../hadoop/hbase/shaded/protobuf/ProtobufUtil.java |  42 +++-
 .../apache/hadoop/hbase/zookeeper/ZNodePaths.java  |  19 +-
 .../org/apache/hadoop/hbase/master/HMaster.java    |  18 +-
 .../hbase/master/MetaRegionLocationCache.java      | 249 +++++++++++++++++++++
 .../hbase/client/TestMetaRegionLocationCache.java  | 186 +++++++++++++++
 .../hbase/master/TestCloseAnOpeningRegion.java     |   5 +-
 .../hbase/master/TestClusterRestartFailover.java   |   2 +-
 .../master/TestRegionsRecoveryConfigManager.java   |   5 +-
 .../hbase/master/TestShutdownBackupMaster.java     |   3 +-
 .../assignment/TestOpenRegionProcedureBackoff.java |   3 +-
 .../assignment/TestOpenRegionProcedureHang.java    |   2 +-
 .../TestRegionAssignedToMultipleRegionServers.java |   3 +-
 .../assignment/TestReportOnlineRegionsRace.java    |   3 +-
 ...tReportRegionStateTransitionFromDeadServer.java |   3 +-
 .../TestReportRegionStateTransitionRetry.java      |   3 +-
 .../master/assignment/TestSCPGetRegionsRace.java   |   3 +-
 .../assignment/TestWakeUpUnexpectedProcedure.java  |   3 +-
 .../TestRegisterPeerWorkerWhenRestarting.java      |   3 +-
 .../hadoop/hbase/protobuf/TestProtobufUtil.java    |  36 ++-
 .../TestRegionServerReportForDuty.java             |   2 +-
 .../replication/TestReplicationProcedureRetry.java |   3 +-
 .../hadoop/hbase/zookeeper/MetaTableLocator.java   |  36 +--
 .../apache/hadoop/hbase/zookeeper/ZKWatcher.java   |  37 ++-
 23 files changed, 586 insertions(+), 83 deletions(-)

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 5a71917..2adcea9 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
@@ -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
@@ -80,6 +80,7 @@ import org.apache.hadoop.hbase.client.PackagePrivateFieldAccessor;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.client.RegionLoadStats;
+import org.apache.hadoop.hbase.client.RegionReplicaUtil;
 import org.apache.hadoop.hbase.client.RegionStatesCount;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
@@ -93,6 +94,7 @@ import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.filter.ByteArrayComparable;
 import org.apache.hadoop.hbase.filter.Filter;
 import org.apache.hadoop.hbase.io.TimeRange;
+import org.apache.hadoop.hbase.master.RegionState;
 import org.apache.hadoop.hbase.protobuf.ProtobufMagic;
 import org.apache.hadoop.hbase.protobuf.ProtobufMessageConverter;
 import org.apache.hadoop.hbase.quotas.QuotaScope;
@@ -3068,6 +3070,44 @@ public final class ProtobufUtil {
   }
 
   /**
+   * Get the Meta region state from the passed data bytes. Can handle both old and new style
+   * server names.
+   * @param data protobuf serialized data with meta server name.
+   * @param replicaId replica ID for this region
+   * @return RegionState instance corresponding to the serialized data.
+   * @throws DeserializationException if the data is invalid.
+   */
+  public static RegionState parseMetaRegionStateFrom(final byte[] data, int replicaId)
+      throws DeserializationException {
+    RegionState.State state = RegionState.State.OPEN;
+    ServerName serverName;
+    if (data != null && data.length > 0 && ProtobufUtil.isPBMagicPrefix(data)) {
+      try {
+        int prefixLen = ProtobufUtil.lengthOfPBMagic();
+        ZooKeeperProtos.MetaRegionServer rl =
+            ZooKeeperProtos.MetaRegionServer.parser().parseFrom(data, prefixLen,
+                data.length - prefixLen);
+        if (rl.hasState()) {
+          state = RegionState.State.convert(rl.getState());
+        }
+        HBaseProtos.ServerName sn = rl.getServer();
+        serverName = ServerName.valueOf(
+            sn.getHostName(), sn.getPort(), sn.getStartCode());
+      } catch (InvalidProtocolBufferException e) {
+        throw new DeserializationException("Unable to parse meta region location");
+      }
+    } else {
+      // old style of meta region location?
+      serverName = parseServerNameFrom(data);
+    }
+    if (serverName == null) {
+      state = RegionState.State.OFFLINE;
+    }
+    return new RegionState(RegionReplicaUtil.getRegionInfoForReplica(
+        RegionInfoBuilder.FIRST_META_REGIONINFO, replicaId), state, serverName);
+  }
+
+  /**
    * Get a ServerName from the passed in data bytes.
    * @param data Data with a serialize server name in it; can handle the old style
    * servername where servername was host and port.  Works too with data that
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZNodePaths.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZNodePaths.java
index 98d7eb7..5c49808 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZNodePaths.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZNodePaths.java
@@ -41,7 +41,8 @@ public class ZNodePaths {
   // TODO: Replace this with ZooKeeper constant when ZOOKEEPER-277 is resolved.
   public static final char ZNODE_PATH_SEPARATOR = '/';
 
-  private static final String META_ZNODE_PREFIX = "meta-region-server";
+  public static final String META_ZNODE_PREFIX_CONF_KEY = "zookeeper.znode.metaserver";
+  public static final String META_ZNODE_PREFIX = "meta-region-server";
   private static final String DEFAULT_SNAPSHOT_CLEANUP_ZNODE = "snapshot-cleanup";
 
   // base znode for this cluster
@@ -104,7 +105,7 @@ public class ZNodePaths {
   public ZNodePaths(Configuration conf) {
     baseZNode = conf.get(ZOOKEEPER_ZNODE_PARENT, DEFAULT_ZOOKEEPER_ZNODE_PARENT);
     ImmutableMap.Builder<Integer, String> builder = ImmutableMap.builder();
-    metaZNodePrefix = conf.get("zookeeper.znode.metaserver", META_ZNODE_PREFIX);
+    metaZNodePrefix = conf.get(META_ZNODE_PREFIX_CONF_KEY, META_ZNODE_PREFIX);
     String defaultMetaReplicaZNode = ZNodePaths.joinZNode(baseZNode, metaZNodePrefix);
     builder.put(DEFAULT_REPLICA_ID, defaultMetaReplicaZNode);
     int numMetaReplicas = conf.getInt(META_REPLICAS_NUM, DEFAULT_META_REPLICA_NUM);
@@ -189,7 +190,19 @@ public class ZNodePaths {
   }
 
   /**
-   * Parse the meta replicaId from the passed znode name.
+   * Parses the meta replicaId from the passed path.
+   * @param path the name of the full path which includes baseZNode.
+   * @return replicaId
+   */
+  public int getMetaReplicaIdFromPath(String path) {
+    // Extract the znode from path. The prefix is of the following format.
+    // baseZNode + PATH_SEPARATOR.
+    int prefixLen = baseZNode.length() + 1;
+    return getMetaReplicaIdFromZnode(path.substring(prefixLen));
+  }
+
+  /**
+   * Parse the meta replicaId from the passed znode
    * @param znode the name of the znode, does not include baseZNode
    * @return replicaId
    */
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 5f5d66d..58f99b5 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
@@ -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
@@ -355,6 +355,12 @@ public class HMaster extends HRegionServer implements MasterServices {
   // manager of assignment nodes in zookeeper
   private AssignmentManager assignmentManager;
 
+  /**
+   * Cache for the meta region replica's locations. Also tracks their changes to avoid stale
+   * cache entries.
+   */
+  private final MetaRegionLocationCache metaRegionLocationCache;
+
   // manager of replication
   private ReplicationPeerManager replicationPeerManager;
 
@@ -508,8 +514,7 @@ public class HMaster extends HRegionServer implements MasterServices {
    * #finishActiveMasterInitialization(MonitoredTask) after
    * the master becomes the active one.
    */
-  public HMaster(final Configuration conf)
-      throws IOException, KeeperException {
+  public HMaster(final Configuration conf) throws IOException {
     super(conf);
     TraceUtil.initTracer(conf);
     try {
@@ -522,7 +527,6 @@ public class HMaster extends HRegionServer implements MasterServices {
       } else {
         maintenanceMode = false;
       }
-
       this.rsFatals = new MemoryBoundedLogMessageBuffer(
           conf.getLong("hbase.master.buffer.for.rs.fatals", 1 * 1024 * 1024));
       LOG.info("hbase.rootdir={}, hbase.cluster.distributed={}", getDataRootDir(),
@@ -570,8 +574,10 @@ public class HMaster extends HRegionServer implements MasterServices {
 
       // Some unit tests don't need a cluster, so no zookeeper at all
       if (!conf.getBoolean("hbase.testing.nocluster", false)) {
+        this.metaRegionLocationCache = new MetaRegionLocationCache(this.zooKeeper);
         this.activeMasterManager = new ActiveMasterManager(zooKeeper, this.serverName, this);
       } else {
+        this.metaRegionLocationCache = null;
         this.activeMasterManager = null;
       }
       cachedClusterId = new CachedClusterId(conf);
@@ -3840,4 +3846,8 @@ public class HMaster extends HRegionServer implements MasterServices {
       rbc.chore();
     }
   }
+
+  public MetaRegionLocationCache getMetaRegionLocationCache() {
+    return this.metaRegionLocationCache;
+  }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java
new file mode 100644
index 0000000..f4e91b5
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java
@@ -0,0 +1,249 @@
+/*
+ * 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.master;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentNavigableMap;
+import java.util.concurrent.ThreadFactory;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.types.CopyOnWriteArrayMap;
+import org.apache.hadoop.hbase.util.RetryCounter;
+import org.apache.hadoop.hbase.util.RetryCounterFactory;
+import org.apache.hadoop.hbase.zookeeper.ZKListener;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+
+/**
+ * A cache of meta region location metadata. Registers a listener on ZK to track changes to the
+ * meta table znodes. Clients are expected to retry if the meta information is stale. This class
+ * is thread-safe (a single instance of this class can be shared by multiple threads without race
+ * conditions).
+ */
+@InterfaceAudience.Private
+public class MetaRegionLocationCache extends ZKListener {
+
+  private static final Logger LOG = LoggerFactory.getLogger(MetaRegionLocationCache.class);
+
+  /**
+   * Maximum number of times we retry when ZK operation times out.
+   */
+  private static final int MAX_ZK_META_FETCH_RETRIES = 10;
+  /**
+   * Sleep interval ms between ZK operation retries.
+   */
+  private static final int SLEEP_INTERVAL_MS_BETWEEN_RETRIES = 1000;
+  private static final int SLEEP_INTERVAL_MS_MAX = 10000;
+  private final RetryCounterFactory retryCounterFactory =
+      new RetryCounterFactory(MAX_ZK_META_FETCH_RETRIES, SLEEP_INTERVAL_MS_BETWEEN_RETRIES);
+
+  /**
+   * Cached meta region locations indexed by replica ID.
+   * CopyOnWriteArrayMap ensures synchronization during updates and a consistent snapshot during
+   * client requests. Even though CopyOnWriteArrayMap copies the data structure for every write,
+   * that should be OK since the size of the list is often small and mutations are not too often
+   * and we do not need to block client requests while mutations are in progress.
+   */
+  private final CopyOnWriteArrayMap<Integer, HRegionLocation> cachedMetaLocations;
+
+  private enum ZNodeOpType {
+    INIT,
+    CREATED,
+    CHANGED,
+    DELETED
+  }
+
+  public MetaRegionLocationCache(ZKWatcher zkWatcher) {
+    super(zkWatcher);
+    cachedMetaLocations = new CopyOnWriteArrayMap<>();
+    watcher.registerListener(this);
+    // Populate the initial snapshot of data from meta znodes.
+    // This is needed because stand-by masters can potentially start after the initial znode
+    // creation. It blocks forever until the initial meta locations are loaded from ZK and watchers
+    // are established. Subsequent updates are handled by the registered listener. Also, this runs
+    // in a separate thread in the background to not block master init.
+    ThreadFactory threadFactory = new ThreadFactoryBuilder().setDaemon(true).build();
+    RetryCounterFactory retryFactory = new RetryCounterFactory(
+        Integer.MAX_VALUE, SLEEP_INTERVAL_MS_BETWEEN_RETRIES, SLEEP_INTERVAL_MS_MAX);
+    threadFactory.newThread(
+      ()->loadMetaLocationsFromZk(retryFactory.create(), ZNodeOpType.INIT)).start();
+  }
+
+  /**
+   * Populates the current snapshot of meta locations from ZK. If no meta znodes exist, it registers
+   * a watcher on base znode to check for any CREATE/DELETE events on the children.
+   * @param retryCounter controls the number of retries and sleep between retries.
+   */
+  private void loadMetaLocationsFromZk(RetryCounter retryCounter, ZNodeOpType opType) {
+    List<String> znodes = null;
+    while (retryCounter.shouldRetry()) {
+      try {
+        znodes = watcher.getMetaReplicaNodesAndWatchChildren();
+        break;
+      } catch (KeeperException ke) {
+        LOG.debug("Error populating initial meta locations", ke);
+        if (!retryCounter.shouldRetry()) {
+          // Retries exhausted and watchers not set. This is not a desirable state since the cache
+          // could remain stale forever. Propagate the exception.
+          watcher.abort("Error populating meta locations", ke);
+          return;
+        }
+        try {
+          retryCounter.sleepUntilNextRetry();
+        } catch (InterruptedException ie) {
+          LOG.error("Interrupted while loading meta locations from ZK", ie);
+          Thread.currentThread().interrupt();
+          return;
+        }
+      }
+    }
+    if (znodes == null || znodes.isEmpty()) {
+      // No meta znodes exist at this point but we registered a watcher on the base znode to listen
+      // for updates. They will be handled via nodeChildrenChanged().
+      return;
+    }
+    if (znodes.size() == cachedMetaLocations.size()) {
+      // No new meta znodes got added.
+      return;
+    }
+    for (String znode: znodes) {
+      String path = ZNodePaths.joinZNode(watcher.getZNodePaths().baseZNode, znode);
+      updateMetaLocation(path, opType);
+    }
+  }
+
+  /**
+   * Gets the HRegionLocation for a given meta replica ID. Renews the watch on the znode for
+   * future updates.
+   * @param replicaId ReplicaID of the region.
+   * @return HRegionLocation for the meta replica.
+   * @throws KeeperException if there is any issue fetching/parsing the serialized data.
+   */
+  private HRegionLocation getMetaRegionLocation(int replicaId)
+      throws KeeperException {
+    RegionState metaRegionState;
+    try {
+      byte[] data = ZKUtil.getDataAndWatch(watcher,
+          watcher.getZNodePaths().getZNodeForReplica(replicaId));
+      metaRegionState = ProtobufUtil.parseMetaRegionStateFrom(data, replicaId);
+    } catch (DeserializationException e) {
+      throw ZKUtil.convert(e);
+    }
+    return new HRegionLocation(metaRegionState.getRegion(), metaRegionState.getServerName());
+  }
+
+  private void updateMetaLocation(String path, ZNodeOpType opType) {
+    if (!isValidMetaZNode(path)) {
+      return;
+    }
+    LOG.debug("Updating meta znode for path {}: {}", path, opType.name());
+    int replicaId = watcher.getZNodePaths().getMetaReplicaIdFromPath(path);
+    RetryCounter retryCounter = retryCounterFactory.create();
+    HRegionLocation location = null;
+    while (retryCounter.shouldRetry()) {
+      try {
+        if (opType == ZNodeOpType.DELETED) {
+          if (!ZKUtil.watchAndCheckExists(watcher, path)) {
+            // The path does not exist, we've set the watcher and we can break for now.
+            break;
+          }
+          // If it is a transient error and the node appears right away, we fetch the
+          // latest meta state.
+        }
+        location = getMetaRegionLocation(replicaId);
+        break;
+      } catch (KeeperException e) {
+        LOG.debug("Error getting meta location for path {}", path, e);
+        if (!retryCounter.shouldRetry()) {
+          LOG.warn("Error getting meta location for path {}. Retries exhausted.", path, e);
+          break;
+        }
+        try {
+          retryCounter.sleepUntilNextRetry();
+        } catch (InterruptedException ie) {
+          Thread.currentThread().interrupt();
+          return;
+        }
+      }
+    }
+    if (location == null) {
+      cachedMetaLocations.remove(replicaId);
+      return;
+    }
+    cachedMetaLocations.put(replicaId, location);
+  }
+
+  /**
+   * @return Optional list of HRegionLocations for meta replica(s), null if the cache is empty.
+   *
+   */
+  public Optional<List<HRegionLocation>> getMetaRegionLocations() {
+    ConcurrentNavigableMap<Integer, HRegionLocation> snapshot =
+        cachedMetaLocations.tailMap(cachedMetaLocations.firstKey());
+    if (snapshot.isEmpty()) {
+      // This could be possible if the master has not successfully initialized yet or meta region
+      // is stuck in some weird state.
+      return Optional.empty();
+    }
+    List<HRegionLocation> result = new ArrayList<>();
+    // Explicitly iterate instead of new ArrayList<>(snapshot.values()) because the underlying
+    // ArrayValueCollection does not implement toArray().
+    snapshot.values().forEach(location -> result.add(location));
+    return Optional.of(result);
+  }
+
+  /**
+   * Helper to check if the given 'path' corresponds to a meta znode. This listener is only
+   * interested in changes to meta znodes.
+   */
+  private boolean isValidMetaZNode(String path) {
+    return watcher.getZNodePaths().isAnyMetaReplicaZNode(path);
+  }
+
+  @Override
+  public void nodeCreated(String path) {
+    updateMetaLocation(path, ZNodeOpType.CREATED);
+  }
+
+  @Override
+  public void nodeDeleted(String path) {
+    updateMetaLocation(path, ZNodeOpType.DELETED);
+  }
+
+  @Override
+  public void nodeDataChanged(String path) {
+    updateMetaLocation(path, ZNodeOpType.CHANGED);
+  }
+
+  @Override
+  public void nodeChildrenChanged(String path) {
+    if (!path.equals(watcher.getZNodePaths().baseZNode)) {
+      return;
+    }
+    loadMetaLocationsFromZk(retryCounterFactory.create(), ZNodeOpType.CHANGED);
+  }
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java
new file mode 100644
index 0000000..02236a6
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java
@@ -0,0 +1,186 @@
+/*
+ * 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.assertFalse;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.MultithreadedTestUtil;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.MetaRegionLocationCache;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.JVMClusterUtil;
+import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({SmallTests.class, MasterTests.class })
+public class TestMetaRegionLocationCache {
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestMetaRegionLocationCache.class);
+
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static AsyncRegistry REGISTRY;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    TEST_UTIL.getConfiguration().setInt(HConstants.META_REPLICAS_NUM, 3);
+    TEST_UTIL.startMiniCluster(3);
+    REGISTRY = AsyncRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
+    RegionReplicaTestHelper.waitUntilAllMetaReplicasHavingRegionLocation(
+        TEST_UTIL.getConfiguration(), REGISTRY, 3);
+    TEST_UTIL.getAdmin().balancerSwitch(false, true);
+  }
+
+  @AfterClass
+  public static void cleanUp() throws Exception {
+    IOUtils.closeQuietly(REGISTRY);
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  private List<HRegionLocation> getCurrentMetaLocations(ZKWatcher zk) throws Exception {
+    List<HRegionLocation> result = new ArrayList<>();
+    for (String znode: zk.getMetaReplicaNodes()) {
+      String path = ZNodePaths.joinZNode(zk.getZNodePaths().baseZNode, znode);
+      int replicaId = zk.getZNodePaths().getMetaReplicaIdFromPath(path);
+      RegionState state = MetaTableLocator.getMetaRegionState(zk, replicaId);
+      result.add(new HRegionLocation(state.getRegion(), state.getServerName()));
+    }
+    return result;
+  }
+
+  // Verifies that the cached meta locations in the given master are in sync with what is in ZK.
+  private void verifyCachedMetaLocations(HMaster master) throws Exception {
+    // Wait until initial meta locations are loaded.
+    int retries = 0;
+    while (!master.getMetaRegionLocationCache().getMetaRegionLocations().isPresent()) {
+      Thread.sleep(1000);
+      if (++retries == 10) {
+        break;
+      }
+    }
+    List<HRegionLocation> metaHRLs =
+        master.getMetaRegionLocationCache().getMetaRegionLocations().get();
+    assertFalse(metaHRLs.isEmpty());
+    ZKWatcher zk = master.getZooKeeper();
+    List<String> metaZnodes = zk.getMetaReplicaNodes();
+    assertEquals(metaZnodes.size(), metaHRLs.size());
+    List<HRegionLocation> actualHRLs = getCurrentMetaLocations(zk);
+    Collections.sort(metaHRLs);
+    Collections.sort(actualHRLs);
+    assertEquals(actualHRLs, metaHRLs);
+  }
+
+  @Test public void testInitialMetaLocations() throws Exception {
+    verifyCachedMetaLocations(TEST_UTIL.getMiniHBaseCluster().getMaster());
+  }
+
+  @Test public void testStandByMetaLocations() throws Exception {
+    HMaster standBy = TEST_UTIL.getMiniHBaseCluster().startMaster().getMaster();
+    verifyCachedMetaLocations(standBy);
+  }
+
+  /*
+   * Shuffles the meta region replicas around the cluster and makes sure the cache is not stale.
+   */
+  @Test public void testMetaLocationsChange() throws Exception {
+    List<HRegionLocation> currentMetaLocs =
+        getCurrentMetaLocations(TEST_UTIL.getMiniHBaseCluster().getMaster().getZooKeeper());
+    // Move these replicas to random servers.
+    for (HRegionLocation location: currentMetaLocs) {
+      RegionReplicaTestHelper.moveRegion(TEST_UTIL, location);
+    }
+    RegionReplicaTestHelper.waitUntilAllMetaReplicasHavingRegionLocation(
+        TEST_UTIL.getConfiguration(), REGISTRY, 3);
+    for (JVMClusterUtil.MasterThread masterThread:
+        TEST_UTIL.getMiniHBaseCluster().getMasterThreads()) {
+      verifyCachedMetaLocations(masterThread.getMaster());
+    }
+  }
+
+  /**
+   * Tests MetaRegionLocationCache's init procedure to make sure that it correctly watches the base
+   * znode for notifications.
+   */
+  @Test public void testMetaRegionLocationCache() throws Exception {
+    final String parentZnodeName = "/randomznodename";
+    Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
+    conf.set(HConstants.ZOOKEEPER_ZNODE_PARENT, parentZnodeName);
+    ServerName sn = ServerName.valueOf("localhost", 1234, 5678);
+    try (ZKWatcher zkWatcher = new ZKWatcher(conf, null, null, true)) {
+      // A thread that repeatedly creates and drops an unrelated child znode. This is to simulate
+      // some ZK activity in the background.
+      MultithreadedTestUtil.TestContext ctx = new MultithreadedTestUtil.TestContext(conf);
+      ctx.addThread(new MultithreadedTestUtil.RepeatingTestThread(ctx) {
+        @Override public void doAnAction() throws Exception {
+          final String testZnode = parentZnodeName + "/child";
+          ZKUtil.createNodeIfNotExistsAndWatch(zkWatcher, testZnode, testZnode.getBytes());
+          ZKUtil.deleteNode(zkWatcher, testZnode);
+        }
+      });
+      ctx.startThreads();
+      try {
+        MetaRegionLocationCache metaCache = new MetaRegionLocationCache(zkWatcher);
+        // meta znodes do not exist at this point, cache should be empty.
+        assertFalse(metaCache.getMetaRegionLocations().isPresent());
+        // Set the meta locations for a random meta replicas, simulating an active hmaster meta
+        // assignment.
+        for (int i = 0; i < 3; i++) {
+          // Updates the meta znodes.
+          MetaTableLocator.setMetaLocation(zkWatcher, sn, i, RegionState.State.OPEN);
+        }
+        // Wait until the meta cache is populated.
+        int iters = 0;
+        while (iters++ < 10) {
+          if (metaCache.getMetaRegionLocations().isPresent()
+            && metaCache.getMetaRegionLocations().get().size() == 3) {
+            break;
+          }
+          Thread.sleep(1000);
+        }
+        List<HRegionLocation> metaLocations = metaCache.getMetaRegionLocations().get();
+        assertEquals(3, metaLocations.size());
+        for (HRegionLocation location : metaLocations) {
+          assertEquals(sn, location.getServerName());
+        }
+      } finally {
+        // clean up.
+        ctx.stop();
+        ZKUtil.deleteChildrenRecursively(zkWatcher, parentZnodeName);
+      }
+    }
+  }
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCloseAnOpeningRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCloseAnOpeningRegion.java
index ba4d535..4922224 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCloseAnOpeningRegion.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCloseAnOpeningRegion.java
@@ -35,7 +35,6 @@ import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.zookeeper.KeeperException;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
@@ -65,7 +64,7 @@ public class TestCloseAnOpeningRegion {
 
   public static final class MockHMaster extends HMaster {
 
-    public MockHMaster(Configuration conf) throws IOException, KeeperException {
+    public MockHMaster(Configuration conf) throws IOException {
       super(conf);
     }
 
@@ -141,4 +140,4 @@ public class TestCloseAnOpeningRegion {
       table.put(new Put(Bytes.toBytes(0)).addColumn(CF, Bytes.toBytes("cq"), Bytes.toBytes(0)));
     }
   }
-}
\ No newline at end of file
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClusterRestartFailover.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClusterRestartFailover.java
index 338173e..1cce2ac 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClusterRestartFailover.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClusterRestartFailover.java
@@ -164,7 +164,7 @@ public class TestClusterRestartFailover extends AbstractTestRestartCluster {
 
   public static final class HMasterForTest extends HMaster {
 
-    public HMasterForTest(Configuration conf) throws IOException, KeeperException {
+    public HMasterForTest(Configuration conf) throws IOException {
       super(conf);
     }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionsRecoveryConfigManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionsRecoveryConfigManager.java
index 22554d3..d29e061 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionsRecoveryConfigManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionsRecoveryConfigManager.java
@@ -28,7 +28,6 @@ import org.apache.hadoop.hbase.StartMiniClusterOption;
 import org.apache.hadoop.hbase.Stoppable;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.zookeeper.KeeperException;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -120,7 +119,7 @@ public class TestRegionsRecoveryConfigManager {
 
   // Make it public so that JVMClusterUtil can access it.
   public static class TestHMaster extends HMaster {
-    public TestHMaster(Configuration conf) throws IOException, KeeperException {
+    public TestHMaster(Configuration conf) throws IOException {
       super(conf);
     }
   }
@@ -144,4 +143,4 @@ public class TestRegionsRecoveryConfigManager {
 
   }
 
-}
\ No newline at end of file
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestShutdownBackupMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestShutdownBackupMaster.java
index d3a8520..a42a404 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestShutdownBackupMaster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestShutdownBackupMaster.java
@@ -30,7 +30,6 @@ import org.apache.hadoop.hbase.StartMiniClusterOption;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
-import org.apache.zookeeper.KeeperException;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
@@ -56,7 +55,7 @@ public class TestShutdownBackupMaster {
 
   public static final class MockHMaster extends HMaster {
 
-    public MockHMaster(Configuration conf) throws IOException, KeeperException {
+    public MockHMaster(Configuration conf) throws IOException {
       super(conf);
     }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureBackoff.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureBackoff.java
index ca0384e..4112da7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureBackoff.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureBackoff.java
@@ -35,7 +35,6 @@ import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.zookeeper.KeeperException;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
@@ -71,7 +70,7 @@ public class TestOpenRegionProcedureBackoff {
 
   public static final class HMasterForTest extends HMaster {
 
-    public HMasterForTest(Configuration conf) throws IOException, KeeperException {
+    public HMasterForTest(Configuration conf) throws IOException {
       super(conf);
     }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureHang.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureHang.java
index 0463721..a25368f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureHang.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureHang.java
@@ -105,7 +105,7 @@ public class TestOpenRegionProcedureHang {
 
   public static final class HMasterForTest extends HMaster {
 
-    public HMasterForTest(Configuration conf) throws IOException, KeeperException {
+    public HMasterForTest(Configuration conf) throws IOException {
       super(conf);
     }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionAssignedToMultipleRegionServers.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionAssignedToMultipleRegionServers.java
index 0d8202b..44af256 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionAssignedToMultipleRegionServers.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionAssignedToMultipleRegionServers.java
@@ -41,7 +41,6 @@ import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.zookeeper.KeeperException;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
@@ -110,7 +109,7 @@ public class TestRegionAssignedToMultipleRegionServers {
 
   public static final class HMasterForTest extends HMaster {
 
-    public HMasterForTest(Configuration conf) throws IOException, KeeperException {
+    public HMasterForTest(Configuration conf) throws IOException {
       super(conf);
     }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportOnlineRegionsRace.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportOnlineRegionsRace.java
index acad88c..4dede89 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportOnlineRegionsRace.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportOnlineRegionsRace.java
@@ -44,7 +44,6 @@ import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.IdLock;
-import org.apache.zookeeper.KeeperException;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
@@ -106,7 +105,7 @@ public class TestReportOnlineRegionsRace {
 
   public static final class HMasterForTest extends HMaster {
 
-    public HMasterForTest(Configuration conf) throws IOException, KeeperException {
+    public HMasterForTest(Configuration conf) throws IOException {
       super(conf);
     }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportRegionStateTransitionFromDeadServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportRegionStateTransitionFromDeadServer.java
index 6c9e5eb..1de806f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportRegionStateTransitionFromDeadServer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportRegionStateTransitionFromDeadServer.java
@@ -44,7 +44,6 @@ import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.zookeeper.KeeperException;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
@@ -117,7 +116,7 @@ public class TestReportRegionStateTransitionFromDeadServer {
 
   public static final class HMasterForTest extends HMaster {
 
-    public HMasterForTest(Configuration conf) throws IOException, KeeperException {
+    public HMasterForTest(Configuration conf) throws IOException {
       super(conf);
     }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportRegionStateTransitionRetry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportRegionStateTransitionRetry.java
index 6c191c9..71c4693 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportRegionStateTransitionRetry.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportRegionStateTransitionRetry.java
@@ -40,7 +40,6 @@ import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.zookeeper.KeeperException;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
@@ -84,7 +83,7 @@ public class TestReportRegionStateTransitionRetry {
 
   public static final class HMasterForTest extends HMaster {
 
-    public HMasterForTest(Configuration conf) throws IOException, KeeperException {
+    public HMasterForTest(Configuration conf) throws IOException {
       super(conf);
     }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestSCPGetRegionsRace.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestSCPGetRegionsRace.java
index cbbdbdc..d676af9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestSCPGetRegionsRace.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestSCPGetRegionsRace.java
@@ -43,7 +43,6 @@ import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.IdLock;
-import org.apache.zookeeper.KeeperException;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
@@ -130,7 +129,7 @@ public class TestSCPGetRegionsRace {
 
   public static final class HMasterForTest extends HMaster {
 
-    public HMasterForTest(Configuration conf) throws IOException, KeeperException {
+    public HMasterForTest(Configuration conf) throws IOException {
       super(conf);
     }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestWakeUpUnexpectedProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestWakeUpUnexpectedProcedure.java
index 47c70a1..62e3161 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestWakeUpUnexpectedProcedure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestWakeUpUnexpectedProcedure.java
@@ -44,7 +44,6 @@ import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
-import org.apache.zookeeper.KeeperException;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
@@ -198,7 +197,7 @@ public class TestWakeUpUnexpectedProcedure {
 
   public static final class HMasterForTest extends HMaster {
 
-    public HMasterForTest(Configuration conf) throws IOException, KeeperException {
+    public HMasterForTest(Configuration conf) throws IOException {
       super(conf);
     }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/TestRegisterPeerWorkerWhenRestarting.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/TestRegisterPeerWorkerWhenRestarting.java
index f46bb41..4dff86d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/TestRegisterPeerWorkerWhenRestarting.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/TestRegisterPeerWorkerWhenRestarting.java
@@ -33,7 +33,6 @@ import org.apache.hadoop.hbase.replication.SyncReplicationTestBase;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
-import org.apache.zookeeper.KeeperException;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.Test;
@@ -53,7 +52,7 @@ public class TestRegisterPeerWorkerWhenRestarting extends SyncReplicationTestBas
 
   public static final class HMasterForTest extends HMaster {
 
-    public HMasterForTest(Configuration conf) throws IOException, KeeperException {
+    public HMasterForTest(Configuration conf) throws IOException {
       super(conf);
     }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java
index ff29df8..69e656f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java
@@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.protobuf;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
-
 import com.google.protobuf.ByteString;
 import java.io.IOException;
 import java.nio.ByteBuffer;
@@ -29,13 +28,17 @@ import org.apache.hadoop.hbase.CellBuilderType;
 import org.apache.hadoop.hbase.CellComparatorImpl;
 import org.apache.hadoop.hbase.ExtendedCellBuilderFactory;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.client.Append;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Increment;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.io.TimeRange;
+import org.apache.hadoop.hbase.master.RegionState;
 import org.apache.hadoop.hbase.protobuf.generated.CellProtos;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Column;
@@ -51,11 +54,12 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.MetaRegionServer;
 
 /**
  * Class to test ProtobufUtil.
  */
-@Category({MiscTests.class, SmallTests.class})
+@Category({ MiscTests.class, SmallTests.class})
 public class TestProtobufUtil {
 
   @ClassRule
@@ -348,4 +352,32 @@ public class TestProtobufUtil {
         ProtobufUtil.toCell(ExtendedCellBuilderFactory.create(CellBuilderType.SHALLOW_COPY), cell);
     assertTrue(CellComparatorImpl.COMPARATOR.compare(offheapKV, newOffheapKV) == 0);
   }
+
+  @Test
+  public void testMetaRegionState() throws Exception {
+    ServerName serverName = ServerName.valueOf("localhost", 1234, 5678);
+    // New region state style.
+    for (RegionState.State state: RegionState.State.values()) {
+      RegionState regionState =
+          new RegionState(RegionInfoBuilder.FIRST_META_REGIONINFO, state, serverName);
+      MetaRegionServer metars = MetaRegionServer.newBuilder()
+          .setServer(org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.toServerName(serverName))
+          .setRpcVersion(HConstants.RPC_CURRENT_VERSION)
+          .setState(state.convert()).build();
+      // Serialize
+      byte[] data = ProtobufUtil.prependPBMagic(metars.toByteArray());
+      ProtobufUtil.prependPBMagic(data);
+      // Deserialize
+      RegionState regionStateNew =
+          org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.parseMetaRegionStateFrom(data, 1);
+      assertEquals(regionState.getServerName(), regionStateNew.getServerName());
+      assertEquals(regionState.getState(), regionStateNew.getState());
+    }
+    // old style.
+    RegionState rs =
+        org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.parseMetaRegionStateFrom(
+            serverName.getVersionedBytes(), 1);
+    assertEquals(serverName, rs.getServerName());
+    assertEquals(rs.getState(), RegionState.State.OPEN);
+  }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerReportForDuty.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerReportForDuty.java
index aaf2d2e..f61a77e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerReportForDuty.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerReportForDuty.java
@@ -122,7 +122,7 @@ public class TestRegionServerReportForDuty {
    * This test HMaster class will always throw ServerNotRunningYetException if checked.
    */
   public static class NeverInitializedMaster extends HMaster {
-    public NeverInitializedMaster(Configuration conf) throws IOException, KeeperException {
+    public NeverInitializedMaster(Configuration conf) throws IOException {
       super(conf);
     }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java
index a2ae0b4..e9fcc66 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java
@@ -36,7 +36,6 @@ import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.replication.ReplicationPeerManager;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
-import org.apache.zookeeper.KeeperException;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -134,7 +133,7 @@ public class TestReplicationProcedureRetry {
 
     private ReplicationPeerManager manager;
 
-    public MockHMaster(Configuration conf) throws IOException, KeeperException {
+    public MockHMaster(Configuration conf) throws IOException {
       super(conf);
     }
 
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java
index 9105d7e..c7b45fe 100644
--- a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java
@@ -34,12 +34,7 @@ import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-
-import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException;
-
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.MetaRegionServer;
 
 /**
@@ -266,40 +261,17 @@ public final class MetaTableLocator {
    * @throws KeeperException if a ZooKeeper operation fails
    */
   public static RegionState getMetaRegionState(ZKWatcher zkw, int replicaId)
-          throws KeeperException {
-    RegionState.State state = RegionState.State.OPEN;
-    ServerName serverName = null;
+      throws KeeperException {
+    RegionState regionState = null;
     try {
       byte[] data = ZKUtil.getData(zkw, zkw.getZNodePaths().getZNodeForReplica(replicaId));
-      if (data != null && data.length > 0 && ProtobufUtil.isPBMagicPrefix(data)) {
-        try {
-          int prefixLen = ProtobufUtil.lengthOfPBMagic();
-          ZooKeeperProtos.MetaRegionServer rl =
-            ZooKeeperProtos.MetaRegionServer.parser().parseFrom(data, prefixLen,
-                    data.length - prefixLen);
-          if (rl.hasState()) {
-            state = RegionState.State.convert(rl.getState());
-          }
-          HBaseProtos.ServerName sn = rl.getServer();
-          serverName = ServerName.valueOf(
-            sn.getHostName(), sn.getPort(), sn.getStartCode());
-        } catch (InvalidProtocolBufferException e) {
-          throw new DeserializationException("Unable to parse meta region location");
-        }
-      } else {
-        // old style of meta region location?
-        serverName = ProtobufUtil.parseServerNameFrom(data);
-      }
+      regionState = ProtobufUtil.parseMetaRegionStateFrom(data, replicaId);
     } catch (DeserializationException e) {
       throw ZKUtil.convert(e);
     } catch (InterruptedException e) {
       Thread.currentThread().interrupt();
     }
-    if (serverName == null) {
-      state = RegionState.State.OFFLINE;
-    }
-    return new RegionState(RegionReplicaUtil.getRegionInfoForReplica(
-        RegionInfoBuilder.FIRST_META_REGIONINFO, replicaId), state, serverName);
+    return regionState;
   }
 
   /**
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKWatcher.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKWatcher.java
index e0b34e4..a39c413 100644
--- a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKWatcher.java
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKWatcher.java
@@ -23,10 +23,8 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.CopyOnWriteArrayList;
-import java.util.concurrent.CountDownLatch;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.AuthUtil;
@@ -81,10 +79,6 @@ public class ZKWatcher implements Watcher, Abortable, Closeable {
   // listeners to be notified
   private final List<ZKListener> listeners = new CopyOnWriteArrayList<>();
 
-  // Used by ZKUtil:waitForZKConnectionIfAuthenticating to wait for SASL
-  // negotiation to complete
-  private CountDownLatch saslLatch = new CountDownLatch(1);
-
   private final Configuration conf;
 
   /* A pattern that matches a Kerberos name, borrowed from Hadoop's KerberosName */
@@ -383,13 +377,32 @@ public class ZKWatcher implements Watcher, Abortable, Closeable {
    */
   public List<String> getMetaReplicaNodes() throws KeeperException {
     List<String> childrenOfBaseNode = ZKUtil.listChildrenNoWatch(this, znodePaths.baseZNode);
+    return filterMetaReplicaNodes(childrenOfBaseNode);
+  }
+
+  /**
+   * Same as {@link #getMetaReplicaNodes()} except that this also registers a watcher on base znode
+   * for subsequent CREATE/DELETE operations on child nodes.
+   */
+  public List<String> getMetaReplicaNodesAndWatchChildren() throws KeeperException {
+    List<String> childrenOfBaseNode =
+        ZKUtil.listChildrenAndWatchForNewChildren(this, znodePaths.baseZNode);
+    return filterMetaReplicaNodes(childrenOfBaseNode);
+  }
+
+  /**
+   * @param nodes Input list of znodes
+   * @return Filtered list of znodes from nodes that belong to meta replica(s).
+   */
+  private List<String> filterMetaReplicaNodes(List<String> nodes) {
+    if (nodes == null || nodes.isEmpty()) {
+      return new ArrayList<>();
+    }
     List<String> metaReplicaNodes = new ArrayList<>(2);
-    if (childrenOfBaseNode != null) {
-      String pattern = conf.get("zookeeper.znode.metaserver","meta-region-server");
-      for (String child : childrenOfBaseNode) {
-        if (child.startsWith(pattern)) {
-          metaReplicaNodes.add(child);
-        }
+    String pattern = conf.get(ZNodePaths.META_ZNODE_PREFIX_CONF_KEY, ZNodePaths.META_ZNODE_PREFIX);
+    for (String child : nodes) {
+      if (child.startsWith(pattern)) {
+        metaReplicaNodes.add(child);
       }
     }
     return metaReplicaNodes;


[hbase] 04/09: HBASE-23604: Clarify AsyncRegistry usage in the code. (#957)

Posted by nd...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

ndimiduk pushed a commit to branch HBASE-18095/client-locate-meta-no-zookeeper
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit 150edac97a6050bc1eb0effcd62cd77b4a4e70f9
Author: Bharath Vissapragada <bh...@apache.org>
AuthorDate: Fri Jan 3 14:27:01 2020 -0800

    HBASE-23604: Clarify AsyncRegistry usage in the code. (#957)
    
    * HBASE-23604: Cleanup AsyncRegistry interface
    
    - Cleans up the method names to make more sense and adds a little
    more javadocs for context. In future patches we can revisit
    the name of the actual class to make it more self explanatory.
    
    - Does AsyncRegistry -> ConnectionRegistry rename.
    "async" ness of the registry is kind of implicit based on
    the interface contents and need not be reflected in the name.
    
    Signed-off-by: Nick Dimiduk <nd...@apache.org>
    Signed-off-by: stack <st...@apache.org>
    Signed-off-by: Viraj Jasani <vj...@apache.org>
---
 .../hadoop/hbase/client/AsyncConnectionImpl.java   | 11 ++++++-----
 .../hbase/client/AsyncMetaRegionLocator.java       |  6 +++---
 .../hbase/client/AsyncTableRegionLocatorImpl.java  |  2 +-
 .../hadoop/hbase/client/ConnectionFactory.java     |  2 +-
 ...{AsyncRegistry.java => ConnectionRegistry.java} | 13 +++++++------
 ...Factory.java => ConnectionRegistryFactory.java} | 22 ++++++++++++----------
 .../hadoop/hbase/client/RawAsyncHBaseAdmin.java    | 17 +++++++++--------
 ...syncRegistry.java => ZKConnectionRegistry.java} | 18 +++++++++---------
 ...istry.java => DoNothingConnectionRegistry.java} |  8 ++++----
 .../hbase/client/TestAsyncAdminRpcPriority.java    |  2 +-
 .../client/TestAsyncMetaRegionLocatorFailFast.java | 10 +++++-----
 .../hbase/client/TestAsyncTableRpcPriority.java    |  2 +-
 ...ryLeak.java => TestConnectionRegistryLeak.java} | 14 +++++++-------
 .../hbase/client/AsyncClusterConnectionImpl.java   |  4 ++--
 .../hbase/client/ClusterConnectionFactory.java     |  2 +-
 .../example/TestZooKeeperTableArchiveClient.java   | 18 +++++++++++-------
 .../hbase/client/AbstractTestRegionLocator.java    |  3 ++-
 ...cRegistry.java => DummyConnectionRegistry.java} | 11 ++++++-----
 .../hbase/client/RegionReplicaTestHelper.java      |  6 +++---
 .../client/TestAsyncAdminWithRegionReplicas.java   |  3 ++-
 .../hbase/client/TestAsyncMetaRegionLocator.java   |  4 ++--
 .../client/TestAsyncNonMetaRegionLocator.java      |  3 ++-
 ...stAsyncNonMetaRegionLocatorConcurrenyLimit.java |  3 ++-
 .../hbase/client/TestAsyncRegionLocator.java       |  3 ++-
 .../TestAsyncSingleRequestRpcRetryingCaller.java   |  3 ++-
 .../client/TestAsyncTableUseMetaReplicas.java      |  2 +-
 .../hbase/client/TestMetaRegionLocationCache.java  |  4 ++--
 ...Registry.java => TestZKConnectionRegistry.java} | 20 ++++++++++----------
 .../regionserver/TestWALEntrySinkFilter.java       | 19 ++++++++-----------
 29 files changed, 124 insertions(+), 111 deletions(-)

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 78fad9e..9d90249 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
@@ -85,7 +85,7 @@ class AsyncConnectionImpl implements AsyncConnection {
 
   private final User user;
 
-  final AsyncRegistry registry;
+  final ConnectionRegistry registry;
 
   private final int rpcTimeout;
 
@@ -122,7 +122,7 @@ class AsyncConnectionImpl implements AsyncConnection {
 
   private volatile ConnectionOverAsyncConnection conn;
 
-  public AsyncConnectionImpl(Configuration conf, AsyncRegistry registry, String clusterId,
+  public AsyncConnectionImpl(Configuration conf, ConnectionRegistry registry, String clusterId,
       SocketAddress localAddress, User user) {
     this.conf = conf;
     this.user = user;
@@ -136,7 +136,8 @@ class AsyncConnectionImpl implements AsyncConnection {
     } else {
       this.metrics = Optional.empty();
     }
-    this.rpcClient = RpcClientFactory.createClient(conf, clusterId, localAddress, metrics.orElse(null));
+    this.rpcClient = RpcClientFactory.createClient(
+        conf, clusterId, localAddress, metrics.orElse(null));
     this.rpcControllerFactory = RpcControllerFactory.instantiate(conf);
     this.hostnameCanChange = conf.getBoolean(RESOLVE_HOSTNAME_ON_FAIL_KEY, true);
     this.rpcTimeout =
@@ -257,7 +258,7 @@ class AsyncConnectionImpl implements AsyncConnection {
   CompletableFuture<MasterService.Interface> getMasterStub() {
     return ConnectionUtils.getOrFetch(masterStub, masterStubMakeFuture, false, () -> {
       CompletableFuture<MasterService.Interface> future = new CompletableFuture<>();
-      addListener(registry.getMasterAddress(), (addr, error) -> {
+      addListener(registry.getActiveMaster(), (addr, error) -> {
         if (error != null) {
           future.completeExceptionally(error);
         } else if (addr == null) {
@@ -368,7 +369,7 @@ class AsyncConnectionImpl implements AsyncConnection {
   @Override
   public CompletableFuture<Hbck> getHbck() {
     CompletableFuture<Hbck> future = new CompletableFuture<>();
-    addListener(registry.getMasterAddress(), (sn, error) -> {
+    addListener(registry.getActiveMaster(), (sn, error) -> {
       if (error != null) {
         future.completeExceptionally(error);
       } else {
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 e920636..3571f960 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
@@ -38,14 +38,14 @@ import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesti
 @InterfaceAudience.Private
 class AsyncMetaRegionLocator {
 
-  private final AsyncRegistry registry;
+  private final ConnectionRegistry registry;
 
   private final AtomicReference<RegionLocations> metaRegionLocations = new AtomicReference<>();
 
   private final AtomicReference<CompletableFuture<RegionLocations>> metaRelocateFuture =
     new AtomicReference<>();
 
-  AsyncMetaRegionLocator(AsyncRegistry registry) {
+  AsyncMetaRegionLocator(ConnectionRegistry registry) {
     this.registry = registry;
   }
 
@@ -60,7 +60,7 @@ class AsyncMetaRegionLocator {
    */
   CompletableFuture<RegionLocations> getRegionLocations(int replicaId, boolean reload) {
     return ConnectionUtils.getOrFetch(metaRegionLocations, metaRelocateFuture, reload,
-      registry::getMetaRegionLocation, locs -> isGood(locs, replicaId), "meta region location");
+      registry::getMetaRegionLocations, locs -> isGood(locs, replicaId), "meta region location");
   }
 
   private HRegionLocation getCacheLocation(HRegionLocation loc) {
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 0e116ab..fa3ea1c 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
@@ -55,7 +55,7 @@ class AsyncTableRegionLocatorImpl implements AsyncTableRegionLocator {
   @Override
   public CompletableFuture<List<HRegionLocation>> getAllRegionLocations() {
     if (TableName.isMetaTableName(tableName)) {
-      return conn.registry.getMetaRegionLocation()
+      return conn.registry.getMetaRegionLocations()
         .thenApply(locs -> Arrays.asList(locs.getRegionLocations()));
     }
     return AsyncMetaTableAccessor.getTableHRegionLocations(conn.getTable(TableName.META_TABLE_NAME),
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 2b432e4..027b898 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
@@ -279,7 +279,7 @@ public class ConnectionFactory {
   public static CompletableFuture<AsyncConnection> createAsyncConnection(Configuration conf,
       final User user) {
     CompletableFuture<AsyncConnection> future = new CompletableFuture<>();
-    AsyncRegistry registry = AsyncRegistryFactory.getRegistry(conf);
+    ConnectionRegistry registry = ConnectionRegistryFactory.getRegistry(conf);
     addListener(registry.getClusterId(), (clusterId, error) -> {
       if (error != null) {
         registry.close();
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegistry.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistry.java
similarity index 77%
rename from hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegistry.java
rename to hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistry.java
index 9537777..cd22d78 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegistry.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistry.java
@@ -24,16 +24,17 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
- * Implementations hold cluster information such as this cluster's id, location of hbase:meta, etc..
+ * Registry for meta information needed for connection setup to a HBase cluster. Implementations
+ * hold cluster information such as this cluster's id, location of hbase:meta, etc..
  * Internal use only.
  */
 @InterfaceAudience.Private
-interface AsyncRegistry extends Closeable {
+interface ConnectionRegistry extends Closeable {
 
   /**
-   * Get the location of meta region.
+   * Get the location of meta region(s).
    */
-  CompletableFuture<RegionLocations> getMetaRegionLocation();
+  CompletableFuture<RegionLocations> getMetaRegionLocations();
 
   /**
    * Should only be called once.
@@ -43,9 +44,9 @@ interface AsyncRegistry extends Closeable {
   CompletableFuture<String> getClusterId();
 
   /**
-   * Get the address of HMaster.
+   * Get the address of active HMaster.
    */
-  CompletableFuture<ServerName> getMasterAddress();
+  CompletableFuture<ServerName> getActiveMaster();
 
   /**
    * Closes this instance and releases any system resources associated with it
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegistryFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistryFactory.java
similarity index 66%
rename from hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegistryFactory.java
rename to hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistryFactory.java
index 28726ae..80d358b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegistryFactory.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistryFactory.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
@@ -18,26 +18,28 @@
 package org.apache.hadoop.hbase.client;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.util.ReflectionUtils;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
- * Get instance of configured Registry.
+ * Factory class to get the instance of configured connection registry.
  */
 @InterfaceAudience.Private
-final class AsyncRegistryFactory {
+final class ConnectionRegistryFactory {
 
-  static final String REGISTRY_IMPL_CONF_KEY = "hbase.client.registry.impl";
+  static final String CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY =
+      "hbase.client.connection.registry.impl";
 
-  private AsyncRegistryFactory() {
+  private ConnectionRegistryFactory() {
   }
 
   /**
-   * @return The cluster registry implementation to use.
+   * @return The connection registry implementation to use.
    */
-  static AsyncRegistry getRegistry(Configuration conf) {
-    Class<? extends AsyncRegistry> clazz =
-        conf.getClass(REGISTRY_IMPL_CONF_KEY, ZKAsyncRegistry.class, AsyncRegistry.class);
+  static ConnectionRegistry getRegistry(Configuration conf) {
+    Class<? extends ConnectionRegistry> clazz = conf.getClass(
+        CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY, ZKConnectionRegistry.class,
+        ConnectionRegistry.class);
     return ReflectionUtils.newInstance(clazz, conf);
   }
 }
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 88022bb..3e5bea3 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
@@ -715,7 +715,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   @Override
   public CompletableFuture<Boolean> isTableAvailable(TableName tableName) {
     if (TableName.isMetaTableName(tableName)) {
-      return connection.registry.getMetaRegionLocation().thenApply(locs -> Stream
+      return connection.registry.getMetaRegionLocations().thenApply(locs -> Stream
         .of(locs.getRegionLocations()).allMatch(loc -> loc != null && loc.getServerName() != null));
     }
     CompletableFuture<Boolean> future = new CompletableFuture<>();
@@ -853,7 +853,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   @Override
   public CompletableFuture<List<RegionInfo>> getRegions(TableName tableName) {
     if (tableName.equals(META_TABLE_NAME)) {
-      return connection.registry.getMetaRegionLocation()
+      return connection.registry.getMetaRegionLocations()
         .thenApply(locs -> Stream.of(locs.getRegionLocations()).map(HRegionLocation::getRegion)
           .collect(Collectors.toList()));
     } else {
@@ -1081,8 +1081,9 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
     if (TableName.META_TABLE_NAME.equals(tableName)) {
       CompletableFuture<List<HRegionLocation>> future = new CompletableFuture<>();
       // For meta table, we use zk to fetch all locations.
-      AsyncRegistry registry = AsyncRegistryFactory.getRegistry(connection.getConfiguration());
-      addListener(registry.getMetaRegionLocation(), (metaRegions, err) -> {
+      ConnectionRegistry registry = ConnectionRegistryFactory.getRegistry(
+          connection.getConfiguration());
+      addListener(registry.getMetaRegionLocations(), (metaRegions, err) -> {
         if (err != null) {
           future.completeExceptionally(err);
         } else if (metaRegions == null || metaRegions.isEmpty() ||
@@ -1110,7 +1111,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
 
     switch (compactType) {
       case MOB:
-        addListener(connection.registry.getMasterAddress(), (serverName, err) -> {
+        addListener(connection.registry.getActiveMaster(), (serverName, err) -> {
           if (err != null) {
             future.completeExceptionally(err);
             return;
@@ -2349,7 +2350,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
         String encodedName = Bytes.toString(regionNameOrEncodedRegionName);
         if (encodedName.length() < RegionInfo.MD5_HEX_LENGTH) {
           // old format encodedName, should be meta region
-          future = connection.registry.getMetaRegionLocation()
+          future = connection.registry.getMetaRegionLocations()
             .thenApply(locs -> Stream.of(locs.getRegionLocations())
               .filter(loc -> loc.getRegion().getEncodedName().equals(encodedName)).findFirst());
         } else {
@@ -2360,7 +2361,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
         RegionInfo regionInfo =
           MetaTableAccessor.parseRegionInfoFromRegionName(regionNameOrEncodedRegionName);
         if (regionInfo.isMetaRegion()) {
-          future = connection.registry.getMetaRegionLocation()
+          future = connection.registry.getMetaRegionLocations()
             .thenApply(locs -> Stream.of(locs.getRegionLocations())
               .filter(loc -> loc.getRegion().getReplicaId() == regionInfo.getReplicaId())
               .findFirst());
@@ -2933,7 +2934,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
 
     switch (compactType) {
       case MOB:
-        addListener(connection.registry.getMasterAddress(), (serverName, err) -> {
+        addListener(connection.registry.getActiveMaster(), (serverName, err) -> {
           if (err != null) {
             future.completeExceptionally(err);
             return;
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKAsyncRegistry.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKConnectionRegistry.java
similarity index 92%
rename from hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKAsyncRegistry.java
rename to hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKConnectionRegistry.java
index b6bacc0..42a4188 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKAsyncRegistry.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKConnectionRegistry.java
@@ -50,15 +50,15 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos;
  * Zookeeper based registry implementation.
  */
 @InterfaceAudience.Private
-class ZKAsyncRegistry implements AsyncRegistry {
+class ZKConnectionRegistry implements ConnectionRegistry {
 
-  private static final Logger LOG = LoggerFactory.getLogger(ZKAsyncRegistry.class);
+  private static final Logger LOG = LoggerFactory.getLogger(ZKConnectionRegistry.class);
 
   private final ReadOnlyZKClient zk;
 
   private final ZNodePaths znodePaths;
 
-  ZKAsyncRegistry(Configuration conf) {
+  ZKConnectionRegistry(Configuration conf) {
     this.znodePaths = new ZNodePaths(conf);
     this.zk = new ReadOnlyZKClient(conf);
   }
@@ -93,7 +93,7 @@ class ZKAsyncRegistry implements AsyncRegistry {
 
   @Override
   public CompletableFuture<String> getClusterId() {
-    return getAndConvert(znodePaths.clusterIdZNode, ZKAsyncRegistry::getClusterId);
+    return getAndConvert(znodePaths.clusterIdZNode, ZKConnectionRegistry::getClusterId);
   }
 
   @VisibleForTesting
@@ -144,7 +144,7 @@ class ZKAsyncRegistry implements AsyncRegistry {
       int replicaId = znodePaths.getMetaReplicaIdFromZnode(metaReplicaZNode);
       String path = ZNodePaths.joinZNode(znodePaths.baseZNode, metaReplicaZNode);
       if (replicaId == DEFAULT_REPLICA_ID) {
-        addListener(getAndConvert(path, ZKAsyncRegistry::getMetaProto), (proto, error) -> {
+        addListener(getAndConvert(path, ZKConnectionRegistry::getMetaProto), (proto, error) -> {
           if (error != null) {
             future.completeExceptionally(error);
             return;
@@ -162,7 +162,7 @@ class ZKAsyncRegistry implements AsyncRegistry {
           tryComplete(remaining, locs, future);
         });
       } else {
-        addListener(getAndConvert(path, ZKAsyncRegistry::getMetaProto), (proto, error) -> {
+        addListener(getAndConvert(path, ZKConnectionRegistry::getMetaProto), (proto, error) -> {
           if (future.isDone()) {
             return;
           }
@@ -191,7 +191,7 @@ class ZKAsyncRegistry implements AsyncRegistry {
   }
 
   @Override
-  public CompletableFuture<RegionLocations> getMetaRegionLocation() {
+  public CompletableFuture<RegionLocations> getMetaRegionLocations() {
     CompletableFuture<RegionLocations> future = new CompletableFuture<>();
     addListener(
       zk.list(znodePaths.baseZNode)
@@ -217,8 +217,8 @@ class ZKAsyncRegistry implements AsyncRegistry {
   }
 
   @Override
-  public CompletableFuture<ServerName> getMasterAddress() {
-    return getAndConvert(znodePaths.masterAddressZNode, ZKAsyncRegistry::getMasterProto)
+  public CompletableFuture<ServerName> getActiveMaster() {
+    return getAndConvert(znodePaths.masterAddressZNode, ZKConnectionRegistry::getMasterProto)
         .thenApply(proto -> {
           if (proto == null) {
             return null;
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/DoNothingAsyncRegistry.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/DoNothingConnectionRegistry.java
similarity index 89%
rename from hbase-client/src/test/java/org/apache/hadoop/hbase/client/DoNothingAsyncRegistry.java
rename to hbase-client/src/test/java/org/apache/hadoop/hbase/client/DoNothingConnectionRegistry.java
index 8c7b073..4bd6687 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/DoNothingAsyncRegistry.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/DoNothingConnectionRegistry.java
@@ -27,13 +27,13 @@ import org.apache.yetus.audience.InterfaceAudience;
  * Registry that does nothing. Otherwise, default Registry wants zookeeper up and running.
  */
 @InterfaceAudience.Private
-class DoNothingAsyncRegistry implements AsyncRegistry {
+class DoNothingConnectionRegistry implements ConnectionRegistry {
 
-  public DoNothingAsyncRegistry(Configuration conf) {
+  public DoNothingConnectionRegistry(Configuration conf) {
   }
 
   @Override
-  public CompletableFuture<RegionLocations> getMetaRegionLocation() {
+  public CompletableFuture<RegionLocations> getMetaRegionLocations() {
     return CompletableFuture.completedFuture(null);
   }
 
@@ -43,7 +43,7 @@ class DoNothingAsyncRegistry implements AsyncRegistry {
   }
 
   @Override
-  public CompletableFuture<ServerName> getMasterAddress() {
+  public CompletableFuture<ServerName> getActiveMaster() {
     return CompletableFuture.completedFuture(null);
   }
 
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminRpcPriority.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminRpcPriority.java
index d4db7a4..75a9433 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminRpcPriority.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminRpcPriority.java
@@ -142,7 +142,7 @@ public class TestAsyncAdminRpcPriority {
     }).when(adminStub).stopServer(any(HBaseRpcController.class), any(StopServerRequest.class),
       any());
 
-    conn = new AsyncConnectionImpl(CONF, new DoNothingAsyncRegistry(CONF), "test", null,
+    conn = new AsyncConnectionImpl(CONF, new DoNothingConnectionRegistry(CONF), "test", null,
       UserProvider.instantiate(CONF).getCurrent()) {
 
       @Override
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocatorFailFast.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocatorFailFast.java
index f29c3bf..b306500 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocatorFailFast.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocatorFailFast.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
@@ -43,21 +43,21 @@ public class TestAsyncMetaRegionLocatorFailFast {
 
   private static AsyncMetaRegionLocator LOCATOR;
 
-  private static final class FaultyAsyncRegistry extends DoNothingAsyncRegistry {
+  private static final class FaultyConnectionRegistry extends DoNothingConnectionRegistry {
 
-    public FaultyAsyncRegistry(Configuration conf) {
+    public FaultyConnectionRegistry(Configuration conf) {
       super(conf);
     }
 
     @Override
-    public CompletableFuture<RegionLocations> getMetaRegionLocation() {
+    public CompletableFuture<RegionLocations> getMetaRegionLocations() {
       return FutureUtils.failedFuture(new DoNotRetryRegionException("inject error"));
     }
   }
 
   @BeforeClass
   public static void setUp() {
-    LOCATOR = new AsyncMetaRegionLocator(new FaultyAsyncRegistry(CONF));
+    LOCATOR = new AsyncMetaRegionLocator(new FaultyConnectionRegistry(CONF));
   }
 
   @Test(expected = DoNotRetryIOException.class)
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRpcPriority.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRpcPriority.java
index 15429cb..b104798 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRpcPriority.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRpcPriority.java
@@ -175,7 +175,7 @@ public class TestAsyncTableRpcPriority {
         return null;
       }
     }).when(stub).get(any(HBaseRpcController.class), any(GetRequest.class), any());
-    conn = new AsyncConnectionImpl(CONF, new DoNothingAsyncRegistry(CONF), "test", null,
+    conn = new AsyncConnectionImpl(CONF, new DoNothingConnectionRegistry(CONF), "test", null,
       UserProvider.instantiate(CONF).getCurrent()) {
 
       @Override
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegistryLeak.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestConnectionRegistryLeak.java
similarity index 83%
rename from hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegistryLeak.java
rename to hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestConnectionRegistryLeak.java
index 7c21075..f02ec42 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegistryLeak.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestConnectionRegistryLeak.java
@@ -38,17 +38,17 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 @Category({ ClientTests.class, SmallTests.class })
-public class TestAsyncRegistryLeak {
+public class TestConnectionRegistryLeak {
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
-    HBaseClassTestRule.forClass(TestAsyncRegistryLeak.class);
+    HBaseClassTestRule.forClass(TestConnectionRegistryLeak.class);
 
-  public static final class AsyncRegistryForTest extends DoNothingAsyncRegistry {
+  public static final class ConnectionRegistryForTest extends DoNothingConnectionRegistry {
 
     private boolean closed = false;
 
-    public AsyncRegistryForTest(Configuration conf) {
+    public ConnectionRegistryForTest(Configuration conf) {
       super(conf);
       CREATED.add(this);
     }
@@ -64,14 +64,14 @@ public class TestAsyncRegistryLeak {
     }
   }
 
-  private static final List<AsyncRegistryForTest> CREATED = new ArrayList<>();
+  private static final List<ConnectionRegistryForTest> CREATED = new ArrayList<>();
 
   private static Configuration CONF = HBaseConfiguration.create();
 
   @BeforeClass
   public static void setUp() {
-    CONF.setClass(AsyncRegistryFactory.REGISTRY_IMPL_CONF_KEY, AsyncRegistryForTest.class,
-      AsyncRegistry.class);
+    CONF.setClass(ConnectionRegistryFactory.CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY,
+        ConnectionRegistryForTest.class, ConnectionRegistry.class);
   }
 
   @Test
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnectionImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnectionImpl.java
index 046ef41..f40f3ac 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnectionImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnectionImpl.java
@@ -50,8 +50,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpeci
 @InterfaceAudience.Private
 class AsyncClusterConnectionImpl extends AsyncConnectionImpl implements AsyncClusterConnection {
 
-  public AsyncClusterConnectionImpl(Configuration conf, AsyncRegistry registry, String clusterId,
-      SocketAddress localAddress, User user) {
+  public AsyncClusterConnectionImpl(Configuration conf, ConnectionRegistry registry,
+      String clusterId, SocketAddress localAddress, User user) {
     super(conf, registry, clusterId, localAddress, user);
   }
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClusterConnectionFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClusterConnectionFactory.java
index 46c0f5a..0094ffa 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClusterConnectionFactory.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClusterConnectionFactory.java
@@ -49,7 +49,7 @@ public final class ClusterConnectionFactory {
    */
   public static AsyncClusterConnection createAsyncClusterConnection(Configuration conf,
       SocketAddress localAddress, User user) throws IOException {
-    AsyncRegistry registry = AsyncRegistryFactory.getRegistry(conf);
+    ConnectionRegistry registry = ConnectionRegistryFactory.getRegistry(conf);
     String clusterId = FutureUtils.get(registry.getClusterId());
     Class<? extends AsyncClusterConnection> clazz =
       conf.getClass(HBASE_SERVER_CLUSTER_CONNECTION_IMPL, AsyncClusterConnectionImpl.class,
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java
index 0945bc9..e8aa822 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java
@@ -21,7 +21,6 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.mock;
-
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
@@ -40,7 +39,7 @@ import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.client.DummyAsyncRegistry;
+import org.apache.hadoop.hbase.client.DummyConnectionRegistry;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.master.cleaner.BaseHFileCleanerDelegate;
 import org.apache.hadoop.hbase.master.cleaner.DirScanPool;
@@ -92,7 +91,7 @@ public class TestZooKeeperTableArchiveClient {
   private static RegionServerServices rss;
   private static DirScanPool POOL;
 
-  public static final class MockRegistry extends DummyAsyncRegistry {
+  public static final class MockRegistry extends DummyConnectionRegistry {
 
     public MockRegistry(Configuration conf) {
     }
@@ -110,8 +109,8 @@ public class TestZooKeeperTableArchiveClient {
   public static void setupCluster() throws Exception {
     setupConf(UTIL.getConfiguration());
     UTIL.startMiniZKCluster();
-    UTIL.getConfiguration().setClass("hbase.client.registry.impl", MockRegistry.class,
-      DummyAsyncRegistry.class);
+    UTIL.getConfiguration().setClass(MockRegistry.REGISTRY_IMPL_CONF_KEY, MockRegistry.class,
+      DummyConnectionRegistry.class);
     CONNECTION = ConnectionFactory.createConnection(UTIL.getConfiguration());
     archivingClient = new ZKTableArchiveClient(UTIL.getConfiguration(), CONNECTION);
     // make hfile archiving node so we can archive files
@@ -147,9 +146,13 @@ public class TestZooKeeperTableArchiveClient {
 
   @AfterClass
   public static void cleanupTest() throws Exception {
-    CONNECTION.close();
+    if (CONNECTION != null) {
+      CONNECTION.close();
+    }
     UTIL.shutdownMiniZKCluster();
-    POOL.shutdownNow();
+    if (POOL != null) {
+      POOL.shutdownNow();
+    }
   }
 
   /**
@@ -353,6 +356,7 @@ public class TestZooKeeperTableArchiveClient {
    * @throws IOException on failure
    * @throws KeeperException on failure
    */
+  @SuppressWarnings("checkstyle:EmptyBlock")
   private List<BaseHFileCleanerDelegate> turnOnArchiving(String tableName, HFileCleaner cleaner)
       throws IOException, KeeperException {
     // turn on hfile retention
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestRegionLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestRegionLocator.java
index 4668d15..f96daf6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestRegionLocator.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestRegionLocator.java
@@ -58,7 +58,8 @@ public abstract class AbstractTestRegionLocator {
     }
     UTIL.getAdmin().createTable(td, SPLIT_KEYS);
     UTIL.waitTableAvailable(TABLE_NAME);
-    try (AsyncRegistry registry = AsyncRegistryFactory.getRegistry(UTIL.getConfiguration())) {
+    try (ConnectionRegistry registry =
+             ConnectionRegistryFactory.getRegistry(UTIL.getConfiguration())) {
       RegionReplicaTestHelper.waitUntilAllMetaReplicasHavingRegionLocation(UTIL.getConfiguration(),
         registry, REGION_REPLICATION);
     }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/DummyAsyncRegistry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/DummyConnectionRegistry.java
similarity index 81%
rename from hbase-server/src/test/java/org/apache/hadoop/hbase/client/DummyAsyncRegistry.java
rename to hbase-server/src/test/java/org/apache/hadoop/hbase/client/DummyConnectionRegistry.java
index 245876e..a669362 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/DummyAsyncRegistry.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/DummyConnectionRegistry.java
@@ -23,14 +23,15 @@ import org.apache.hadoop.hbase.ServerName;
 
 /**
  * Can be overridden in UT if you only want to implement part of the methods in
- * {@link AsyncRegistry}.
+ * {@link ConnectionRegistry}.
  */
-public class DummyAsyncRegistry implements AsyncRegistry {
+public class DummyConnectionRegistry implements ConnectionRegistry {
 
-  public static final String REGISTRY_IMPL_CONF_KEY = AsyncRegistryFactory.REGISTRY_IMPL_CONF_KEY;
+  public static final String REGISTRY_IMPL_CONF_KEY =
+      ConnectionRegistryFactory.CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY;
 
   @Override
-  public CompletableFuture<RegionLocations> getMetaRegionLocation() {
+  public CompletableFuture<RegionLocations> getMetaRegionLocations() {
     return null;
   }
 
@@ -40,7 +41,7 @@ public class DummyAsyncRegistry implements AsyncRegistry {
   }
 
   @Override
-  public CompletableFuture<ServerName> getMasterAddress() {
+  public CompletableFuture<ServerName> getActiveMaster() {
     return null;
   }
 
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
index ba3e74c..73924a3 100644
--- 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
@@ -44,7 +44,7 @@ public final class RegionReplicaTestHelper {
 
   // waits for all replicas to have region location
   static void waitUntilAllMetaReplicasHavingRegionLocation(Configuration conf,
-      AsyncRegistry registry, int regionReplication) throws IOException {
+      ConnectionRegistry registry, int regionReplication) throws IOException {
     Waiter.waitFor(conf, conf.getLong("hbase.client.sync.wait.timeout.msec", 60000), 200, true,
       new ExplainingPredicate<IOException>() {
         @Override
@@ -55,7 +55,7 @@ public final class RegionReplicaTestHelper {
         @Override
         public boolean evaluate() throws IOException {
           try {
-            RegionLocations locs = registry.getMetaRegionLocation().get();
+            RegionLocations locs = registry.getMetaRegionLocations().get();
             if (locs.size() < regionReplication) {
               return false;
             }
@@ -66,7 +66,7 @@ public final class RegionReplicaTestHelper {
             }
             return true;
           } catch (Exception e) {
-            TestZKAsyncRegistry.LOG.warn("Failed to get meta region locations", e);
+            TestZKConnectionRegistry.LOG.warn("Failed to get meta region locations", e);
             return false;
           }
         }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminWithRegionReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminWithRegionReplicas.java
index 1401afe..6d7d368 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminWithRegionReplicas.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminWithRegionReplicas.java
@@ -53,7 +53,8 @@ public class TestAsyncAdminWithRegionReplicas extends TestAsyncAdminBase {
   public static void setUpBeforeClass() throws Exception {
     TEST_UTIL.getConfiguration().setInt(HConstants.META_REPLICAS_NUM, 3);
     TestAsyncAdminBase.setUpBeforeClass();
-    try (AsyncRegistry registry = AsyncRegistryFactory.getRegistry(TEST_UTIL.getConfiguration())) {
+    try (ConnectionRegistry registry =
+             ConnectionRegistryFactory.getRegistry(TEST_UTIL.getConfiguration())) {
       RegionReplicaTestHelper
         .waitUntilAllMetaReplicasHavingRegionLocation(TEST_UTIL.getConfiguration(), registry, 3);
     }
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 1f0d40b..609a129 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
@@ -45,7 +45,7 @@ public class TestAsyncMetaRegionLocator {
 
   private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 
-  private static AsyncRegistry REGISTRY;
+  private static ConnectionRegistry REGISTRY;
 
   private static AsyncMetaRegionLocator LOCATOR;
 
@@ -54,7 +54,7 @@ public class TestAsyncMetaRegionLocator {
     TEST_UTIL.getConfiguration().set(BaseLoadBalancer.TABLES_ON_MASTER, "none");
     TEST_UTIL.getConfiguration().setInt(HConstants.META_REPLICAS_NUM, 3);
     TEST_UTIL.startMiniCluster(3);
-    REGISTRY = AsyncRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
+    REGISTRY = ConnectionRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
     RegionReplicaTestHelper
       .waitUntilAllMetaReplicasHavingRegionLocation(TEST_UTIL.getConfiguration(), REGISTRY, 3);
     TEST_UTIL.getAdmin().balancerSwitch(false, true);
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 d1ed5b7..d8388de 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
@@ -79,7 +79,8 @@ public class TestAsyncNonMetaRegionLocator {
   public static void setUp() throws Exception {
     TEST_UTIL.startMiniCluster(3);
     TEST_UTIL.getAdmin().balancerSwitch(false, true);
-    AsyncRegistry registry = AsyncRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
+    ConnectionRegistry registry =
+        ConnectionRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
     CONN = new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), registry,
       registry.getClusterId().get(), null, User.getCurrent());
     LOCATOR = new AsyncNonMetaRegionLocator(CONN);
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 7e06218..88ab3ad 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
@@ -123,7 +123,8 @@ public class TestAsyncNonMetaRegionLocatorConcurrenyLimit {
     conf.setInt(MAX_CONCURRENT_LOCATE_REQUEST_PER_TABLE, MAX_ALLOWED);
     TEST_UTIL.startMiniCluster(3);
     TEST_UTIL.getAdmin().balancerSwitch(false, true);
-    AsyncRegistry registry = AsyncRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
+    ConnectionRegistry registry =
+        ConnectionRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
     CONN = new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), registry,
       registry.getClusterId().get(), null, User.getCurrent());
     LOCATOR = new AsyncNonMetaRegionLocator(CONN);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocator.java
index a7cf322..753e4f3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocator.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocator.java
@@ -98,7 +98,8 @@ public class TestAsyncRegionLocator {
     TEST_UTIL.startMiniCluster(1);
     TEST_UTIL.createTable(TABLE_NAME, FAMILY);
     TEST_UTIL.waitTableAvailable(TABLE_NAME);
-    AsyncRegistry registry = AsyncRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
+    ConnectionRegistry registry =
+        ConnectionRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
     CONN = new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), registry,
       registry.getClusterId().get(), null, User.getCurrent());
     LOCATOR = CONN.getLocator();
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 b6e5362..4205012 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
@@ -71,7 +71,8 @@ public class TestAsyncSingleRequestRpcRetryingCaller {
     TEST_UTIL.getAdmin().balancerSwitch(false, true);
     TEST_UTIL.createTable(TABLE_NAME, FAMILY);
     TEST_UTIL.waitTableAvailable(TABLE_NAME);
-    AsyncRegistry registry = AsyncRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
+    ConnectionRegistry registry =
+        ConnectionRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
     CONN = new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), registry,
       registry.getClusterId().get(), null, User.getCurrent());
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableUseMetaReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableUseMetaReplicas.java
index e74d3fa..ed6c66f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableUseMetaReplicas.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableUseMetaReplicas.java
@@ -92,7 +92,7 @@ public class TestAsyncTableUseMetaReplicas {
     conf.setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,
       FailPrimaryMetaScanCp.class.getName());
     UTIL.startMiniCluster(3);
-    try (AsyncRegistry registry = AsyncRegistryFactory.getRegistry(conf)) {
+    try (ConnectionRegistry registry = ConnectionRegistryFactory.getRegistry(conf)) {
       RegionReplicaTestHelper.waitUntilAllMetaReplicasHavingRegionLocation(conf, registry, 3);
     }
     try (Table table = UTIL.createTable(TABLE_NAME, FAMILY)) {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java
index 02236a6..1205b05 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java
@@ -53,13 +53,13 @@ public class TestMetaRegionLocationCache {
       HBaseClassTestRule.forClass(TestMetaRegionLocationCache.class);
 
   private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-  private static AsyncRegistry REGISTRY;
+  private static ConnectionRegistry REGISTRY;
 
   @BeforeClass
   public static void setUp() throws Exception {
     TEST_UTIL.getConfiguration().setInt(HConstants.META_REPLICAS_NUM, 3);
     TEST_UTIL.startMiniCluster(3);
-    REGISTRY = AsyncRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
+    REGISTRY = ConnectionRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
     RegionReplicaTestHelper.waitUntilAllMetaReplicasHavingRegionLocation(
         TEST_UTIL.getConfiguration(), REGISTRY, 3);
     TEST_UTIL.getAdmin().balancerSwitch(false, true);
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/TestZKConnectionRegistry.java
similarity index 89%
rename from hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKAsyncRegistry.java
rename to hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKConnectionRegistry.java
index 3e4ca94..cb45ee5 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/TestZKConnectionRegistry.java
@@ -48,16 +48,16 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @Category({ MediumTests.class, ClientTests.class })
-public class TestZKAsyncRegistry {
+public class TestZKConnectionRegistry {
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
-    HBaseClassTestRule.forClass(TestZKAsyncRegistry.class);
+    HBaseClassTestRule.forClass(TestZKConnectionRegistry.class);
 
-  static final Logger LOG = LoggerFactory.getLogger(TestZKAsyncRegistry.class);
+  static final Logger LOG = LoggerFactory.getLogger(TestZKConnectionRegistry.class);
   static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 
-  private static ZKAsyncRegistry REGISTRY;
+  private static ZKConnectionRegistry REGISTRY;
 
   @BeforeClass
   public static void setUp() throws Exception {
@@ -67,7 +67,7 @@ public class TestZKAsyncRegistry {
     // make sure that we do not depend on this config when getting locations for meta replicas, see
     // HBASE-21658.
     conf.setInt(META_REPLICAS_NUM, 1);
-    REGISTRY = new ZKAsyncRegistry(conf);
+    REGISTRY = new ZKConnectionRegistry(conf);
   }
 
   @AfterClass
@@ -84,10 +84,10 @@ public class TestZKAsyncRegistry {
     assertEquals("Expected " + expectedClusterId + ", found=" + clusterId, expectedClusterId,
       clusterId);
     assertEquals(TEST_UTIL.getHBaseCluster().getMaster().getServerName(),
-      REGISTRY.getMasterAddress().get());
+      REGISTRY.getActiveMaster().get());
     RegionReplicaTestHelper
       .waitUntilAllMetaReplicasHavingRegionLocation(TEST_UTIL.getConfiguration(), REGISTRY, 3);
-    RegionLocations locs = REGISTRY.getMetaRegionLocation().get();
+    RegionLocations locs = REGISTRY.getMetaRegionLocations().get();
     assertEquals(3, locs.getRegionLocations().length);
     IntStream.range(0, 3).forEach(i -> {
       HRegionLocation loc = locs.getRegionLocation(i);
@@ -102,7 +102,7 @@ public class TestZKAsyncRegistry {
     try (ReadOnlyZKClient zk1 = REGISTRY.getZKClient()) {
       Configuration otherConf = new Configuration(TEST_UTIL.getConfiguration());
       otherConf.set(HConstants.ZOOKEEPER_QUORUM, "127.0.0.1");
-      try (ZKAsyncRegistry otherRegistry = new ZKAsyncRegistry(otherConf)) {
+      try (ZKConnectionRegistry otherRegistry = new ZKConnectionRegistry(otherConf)) {
         ReadOnlyZKClient zk2 = otherRegistry.getZKClient();
         assertNotSame("Using a different configuration / quorum should result in different " +
           "backing zk connection.", zk1, zk2);
@@ -119,9 +119,9 @@ public class TestZKAsyncRegistry {
   public void testNoMetaAvailable() throws InterruptedException {
     Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
     conf.set("zookeeper.znode.metaserver", "whatever");
-    try (ZKAsyncRegistry registry = new ZKAsyncRegistry(conf)) {
+    try (ZKConnectionRegistry registry = new ZKConnectionRegistry(conf)) {
       try {
-        registry.getMetaRegionLocation().get();
+        registry.getMetaRegionLocations().get();
         fail("Should have failed since we set an incorrect meta znode prefix");
       } catch (ExecutionException e) {
         assertThat(e.getCause(), instanceOf(IOException.class));
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntrySinkFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntrySinkFilter.java
index 41d4f46..5e16723 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntrySinkFilter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntrySinkFilter.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
@@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.replication.regionserver;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
-
 import java.io.IOException;
 import java.net.SocketAddress;
 import java.util.ArrayList;
@@ -43,8 +42,8 @@ import org.apache.hadoop.hbase.client.AsyncConnection;
 import org.apache.hadoop.hbase.client.AsyncTable;
 import org.apache.hadoop.hbase.client.ClusterConnectionFactory;
 import org.apache.hadoop.hbase.client.DummyAsyncClusterConnection;
-import org.apache.hadoop.hbase.client.DummyAsyncRegistry;
 import org.apache.hadoop.hbase.client.DummyAsyncTable;
+import org.apache.hadoop.hbase.client.DummyConnectionRegistry;
 import org.apache.hadoop.hbase.client.Row;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
@@ -57,9 +56,7 @@ import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-
 import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
-
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
 
 /**
@@ -108,12 +105,12 @@ public class TestWALEntrySinkFilter {
   public void testWALEntryFilter() throws IOException {
     Configuration conf = HBaseConfiguration.create();
     // Make it so our filter is instantiated on construction of ReplicationSink.
-    conf.setClass(DummyAsyncRegistry.REGISTRY_IMPL_CONF_KEY, DevNullAsyncRegistry.class,
-      DummyAsyncRegistry.class);
+    conf.setClass(DummyConnectionRegistry.REGISTRY_IMPL_CONF_KEY, DevNullConnectionRegistry.class,
+        DummyConnectionRegistry.class);
     conf.setClass(WALEntrySinkFilter.WAL_ENTRY_FILTER_KEY,
-      IfTimeIsGreaterThanBOUNDARYWALEntrySinkFilterImpl.class, WALEntrySinkFilter.class);
+        IfTimeIsGreaterThanBOUNDARYWALEntrySinkFilterImpl.class, WALEntrySinkFilter.class);
     conf.setClass(ClusterConnectionFactory.HBASE_SERVER_CLUSTER_CONNECTION_IMPL,
-      DevNullAsyncClusterConnection.class, AsyncClusterConnection.class);
+        DevNullAsyncClusterConnection.class, AsyncClusterConnection.class);
     ReplicationSink sink = new ReplicationSink(conf, STOPPABLE);
     // Create some dumb walentries.
     List<AdminProtos.WALEntry> entries = new ArrayList<>();
@@ -190,9 +187,9 @@ public class TestWALEntrySinkFilter {
     }
   }
 
-  public static class DevNullAsyncRegistry extends DummyAsyncRegistry {
+  public static class DevNullConnectionRegistry extends DummyConnectionRegistry {
 
-    public DevNullAsyncRegistry(Configuration conf) {
+    public DevNullConnectionRegistry(Configuration conf) {
     }
 
     @Override


[hbase] 09/09: HBASE-23731: De-flake TestFromClientSide (#1091)

Posted by nd...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

ndimiduk pushed a commit to branch HBASE-18095/client-locate-meta-no-zookeeper
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit 60cb4f35aeddf07609be92d92a3c8fddf88909c1
Author: Bharath Vissapragada <bh...@apache.org>
AuthorDate: Mon Jan 27 12:37:02 2020 -0800

    HBASE-23731: De-flake TestFromClientSide (#1091)
    
    There were a couple of issues.
    
    - There was a leak of a file descriptor for hbck lock file. This
    was contributing to all the "ConnectionRefused" stack traces since
    it was trying to renew lease for an already expired mini dfs cluster.
    This issue was there for a while, just that we noticed it now.
    
    - After upgrade to JUnit 4.13, it looks like the behavior for test
    timeouts has changed. Earlier the timeout seems to have applied for
    each parameterized run, but now it looks like it is applied across
    all the runs.
    
    This patch fixes both the issues.
    
    Signed-off-by: Stack <st...@apache.org>
    Signed-off-by: Jan Hentschel <ja...@ultratendency.com>
---
 .../apache/hadoop/hbase/HBaseClassTestRule.java    |  84 +++++++++++-
 .../hadoop/hbase/TestHBaseClassTestRule.java       | 145 +++++++++++++++++++++
 .../org/apache/hadoop/hbase/master/HMaster.java    |  16 ++-
 3 files changed, 234 insertions(+), 11 deletions(-)

diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRule.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRule.java
index 00374c1..0880ad0 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRule.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRule.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
@@ -17,10 +17,16 @@
  */
 package org.apache.hadoop.hbase;
 
+import edu.umd.cs.findbugs.annotations.NonNull;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
+import java.util.List;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
-
 import org.apache.hadoop.hbase.testclassification.IntegrationTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -30,8 +36,14 @@ import org.junit.experimental.categories.Category;
 import org.junit.rules.TestRule;
 import org.junit.rules.Timeout;
 import org.junit.runner.Description;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
 import org.junit.runners.model.Statement;
-
+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.collect.Iterables;
 import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
 
 /**
@@ -43,9 +55,13 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
  */
 @InterfaceAudience.Private
 public final class HBaseClassTestRule implements TestRule {
+  private static final Logger LOG = LoggerFactory.getLogger(HBaseClassTestRule.class);
   public static final Set<Class<?>> UNIT_TEST_CLASSES = Collections.unmodifiableSet(
       Sets.<Class<?>> newHashSet(SmallTests.class, MediumTests.class, LargeTests.class));
 
+  // Each unit test has this timeout.
+  private static long PER_UNIT_TEST_TIMEOUT_MINS = 13;
+
   private final Class<?> clazz;
 
   private final Timeout timeout;
@@ -65,13 +81,16 @@ public final class HBaseClassTestRule implements TestRule {
 
   private static long getTimeoutInSeconds(Class<?> clazz) {
     Category[] categories = clazz.getAnnotationsByType(Category.class);
-
+    // Starting JUnit 4.13, it appears that the timeout is applied across all the parameterized
+    // runs. So the timeout is multiplied by number of parameterized runs.
+    int numParams = getNumParameters(clazz);
     // @Category is not repeatable -- it is only possible to get an array of length zero or one.
     if (categories.length == 1) {
       for (Class<?> c : categories[0].value()) {
         if (UNIT_TEST_CLASSES.contains(c)) {
-          // All tests have a 13 minutes timeout.
-          return TimeUnit.MINUTES.toSeconds(13);
+          long timeout = numParams * PER_UNIT_TEST_TIMEOUT_MINS;
+          LOG.info("Test {} timeout: {} mins", clazz, timeout);
+          return TimeUnit.MINUTES.toSeconds(timeout);
         }
         if (c == IntegrationTests.class) {
           return TimeUnit.MINUTES.toSeconds(Long.MAX_VALUE);
@@ -82,6 +101,59 @@ public final class HBaseClassTestRule implements TestRule {
         clazz.getName() + " does not have SmallTests/MediumTests/LargeTests in @Category");
   }
 
+  /**
+   * @param clazz Test class that is running.
+   * @return the number of parameters for this given test class. If the test is not parameterized or
+   *   if there is any issue determining the number of parameters, returns 1.
+   */
+  @VisibleForTesting
+  static int getNumParameters(Class<?> clazz) {
+    RunWith[] runWiths = clazz.getAnnotationsByType(RunWith.class);
+    boolean testParameterized = runWiths != null && Arrays.stream(runWiths).anyMatch(
+      (r) -> r.value().equals(Parameterized.class));
+    if (!testParameterized) {
+      return 1;
+    }
+    for (Method method : clazz.getMethods()) {
+      if (!isParametersMethod(method)) {
+        continue;
+      }
+      // Found the parameters method. Figure out the number of parameters.
+      Object parameters;
+      try {
+        parameters = method.invoke(clazz);
+      } catch (IllegalAccessException | InvocationTargetException e) {
+        LOG.warn("Error invoking parameters method {} in test class {}",
+            method.getName(), clazz, e);
+        continue;
+      }
+      if (parameters instanceof List) {
+        return  ((List) parameters).size();
+      } else if (parameters instanceof Collection) {
+        return  ((Collection) parameters).size();
+      } else if (parameters instanceof Iterable) {
+        return Iterables.size((Iterable) parameters);
+      } else if (parameters instanceof Object[]) {
+        return ((Object[]) parameters).length;
+      }
+    }
+    LOG.warn("Unable to determine parameters size. Returning the default of 1.");
+    return 1;
+  }
+
+  /**
+   * Helper method that checks if the input method is a valid JUnit @Parameters method.
+   * @param method Input method.
+   * @return true if the method is a valid JUnit parameters method, false otherwise.
+   */
+  private static boolean isParametersMethod(@NonNull Method method) {
+    // A valid parameters method is public static and with @Parameters annotation.
+    boolean methodPublicStatic = Modifier.isPublic(method.getModifiers()) &&
+        Modifier.isStatic(method.getModifiers());
+    Parameters[] params = method.getAnnotationsByType(Parameters.class);
+    return methodPublicStatic && (params != null && params.length > 0);
+  }
+
   public static HBaseClassTestRule forClass(Class<?> clazz) {
     return new HBaseClassTestRule(clazz, Timeout.builder().withLookingForStuckThread(true)
         .withTimeout(getTimeoutInSeconds(clazz), TimeUnit.SECONDS).build());
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestHBaseClassTestRule.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestHBaseClassTestRule.java
new file mode 100644
index 0000000..78853e6
--- /dev/null
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestHBaseClassTestRule.java
@@ -0,0 +1,145 @@
+/*
+ * 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;
+
+import static junit.framework.TestCase.assertEquals;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+import org.apache.hbase.thirdparty.com.google.common.collect.Iterables;
+
+/**
+ * Tests HBaseClassTestRule.
+ */
+@Category(SmallTests.class)
+public class TestHBaseClassTestRule {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE = HBaseClassTestRule.forClass(
+      TestHBaseClassTestRule.class);
+
+  // Test input classes of various kinds.
+  private static class NonParameterizedClass {
+    void dummy() {
+    }
+    int dummy(int a) {
+      return 0;
+    }
+  }
+
+  @RunWith(Parameterized.class)
+  private static class ParameterizedClassWithNoParametersMethod {
+    void dummy() {
+    }
+  }
+
+  @RunWith(Parameterized.class)
+  private static class InValidParameterizedClass {
+    // Not valid because parameters method is private.
+    @Parameters
+    private static List<Object> parameters() {
+      return Arrays.asList(1, 2, 3, 4);
+    }
+    int dummy(int a) {
+      return 0;
+    }
+  }
+
+  @RunWith(Parameterized.class)
+  private static class ValidParameterizedClass1 {
+    @Parameters
+    public static List<Object> parameters() {
+      return Arrays.asList(1, 2, 3, 4, 5);
+    }
+    int dummy(int a) {
+      return 0;
+    }
+  }
+
+  @RunWith(Parameterized.class)
+  private static class ValidParameterizedClass2 {
+    @Parameters
+    public static Object[] parameters() {
+      return new Integer[] {1, 2, 3, 4, 5, 6};
+    }
+  }
+
+  @RunWith(Parameterized.class)
+  private static class ValidParameterizedClass3 {
+    @Parameters
+    public static Iterable<Integer> parameters() {
+      return Arrays.asList(1, 2, 3, 4, 5, 6, 7);
+    }
+  }
+
+  @RunWith(Parameterized.class)
+  private static class ValidParameterizedClass4 {
+    @Parameters
+    public static Collection<Integer> parameters() {
+      return Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9);
+    }
+  }
+
+
+  @RunWith(Parameterized.class)
+  private static class ExtendedParameterizedClass1 extends ValidParameterizedClass1 {
+    // Should be inferred from the parent class.
+    int dummy2(int a) {
+      return 0;
+    }
+  }
+
+  @RunWith(Parameterized.class)
+  private static class ExtendedParameterizedClass2 extends ValidParameterizedClass1 {
+    // Should override the parent parameters class.
+    @Parameters
+    public static List<Object> parameters() {
+      return Arrays.asList(1, 2, 3);
+    }
+  }
+
+  @Test
+  public void testNumParameters() {
+    // Invalid cases, expected to return 1.
+    assertEquals(HBaseClassTestRule.getNumParameters(NonParameterizedClass.class), 1);
+    assertEquals(HBaseClassTestRule.getNumParameters(
+        ParameterizedClassWithNoParametersMethod.class), 1);
+    assertEquals(HBaseClassTestRule.getNumParameters(InValidParameterizedClass.class), 1);
+    // Valid parameterized classes.
+    assertEquals(HBaseClassTestRule.getNumParameters(ValidParameterizedClass1.class),
+        ValidParameterizedClass1.parameters().size());
+    assertEquals(HBaseClassTestRule.getNumParameters(ValidParameterizedClass2.class),
+        ValidParameterizedClass2.parameters().length);
+    assertEquals(HBaseClassTestRule.getNumParameters(ValidParameterizedClass3.class),
+        Iterables.size(ValidParameterizedClass3.parameters()));
+    assertEquals(HBaseClassTestRule.getNumParameters(ValidParameterizedClass4.class),
+        ValidParameterizedClass4.parameters().size());
+    // Testing inheritance.
+    assertEquals(HBaseClassTestRule.getNumParameters(ExtendedParameterizedClass1.class),
+        ValidParameterizedClass1.parameters().size());
+    assertEquals(HBaseClassTestRule.getNumParameters(ExtendedParameterizedClass2.class),
+        ExtendedParameterizedClass2.parameters().size());
+  }
+}
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 def82b4..0b6f994 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
@@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.master;
 import static org.apache.hadoop.hbase.HConstants.DEFAULT_HBASE_SPLIT_COORDINATED_BY_ZK;
 import static org.apache.hadoop.hbase.HConstants.HBASE_MASTER_LOGCLEANER_PLUGINS;
 import static org.apache.hadoop.hbase.HConstants.HBASE_SPLIT_WAL_COORDINATED_BY_ZK;
-
 import com.google.protobuf.Descriptors;
 import com.google.protobuf.Service;
 import java.io.IOException;
@@ -56,8 +55,10 @@ import javax.servlet.ServletException;
 import javax.servlet.http.HttpServlet;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
+import org.apache.commons.io.IOUtils;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.ChoreService;
 import org.apache.hadoop.hbase.ClusterId;
@@ -219,11 +220,9 @@ import org.eclipse.jetty.servlet.ServletHolder;
 import org.eclipse.jetty.webapp.WebAppContext;
 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.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;
@@ -922,8 +921,15 @@ public class HMaster extends HRegionServer implements MasterServices {
     // hbck1s against an hbase2 cluster; it could do damage. To skip this behavior, set
     // hbase.write.hbck1.lock.file to false.
     if (this.conf.getBoolean("hbase.write.hbck1.lock.file", true)) {
-      HBaseFsck.checkAndMarkRunningHbck(this.conf,
-          HBaseFsck.createLockRetryCounterFactory(this.conf).create());
+      Pair<Path, FSDataOutputStream> result = null;
+      try {
+        result = HBaseFsck.checkAndMarkRunningHbck(this.conf,
+            HBaseFsck.createLockRetryCounterFactory(this.conf).create());
+      } finally {
+        if (result != null) {
+          IOUtils.closeQuietly(result.getSecond());
+        }
+      }
     }
 
     status.setStatus("Initialize ServerManager and schedule SCP for crash servers");


[hbase] 08/09: HBASE-23647: Make MasterRegistry the default impl. (#1039)

Posted by nd...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

ndimiduk pushed a commit to branch HBASE-18095/client-locate-meta-no-zookeeper
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit 89c02356358365e0b9e2d57409aaa3c48ed75382
Author: Bharath Vissapragada <bh...@apache.org>
AuthorDate: Mon Jan 27 12:36:09 2020 -0800

    HBASE-23647: Make MasterRegistry the default impl. (#1039)
    
    Signed-off-by: Stack <st...@apache.org>
    Signed-off-by: Nick Dimiduk <nd...@apache.org>
    Signed-off-by: Andrew Purtell <ap...@apache.org>
---
 .../hbase/client/ConnectionRegistryFactory.java    |   2 +-
 .../apache/hadoop/hbase/client/MasterRegistry.java |   4 +-
 .../replication/ReplicationPeerConfigUtil.java     |   1 -
 .../apache/hadoop/hbase/security/SecurityInfo.java |   2 +
 .../apache/hadoop/hbase/HBaseConfiguration.java    |  13 ++-
 .../java/org/apache/hadoop/hbase/HConstants.java   |   4 +
 .../org/apache/hadoop/hbase/LocalHBaseCluster.java |  43 +++++----
 .../hadoop/hbase/master/ActiveMasterManager.java   |  10 +-
 .../org/apache/hadoop/hbase/master/HMaster.java    |  11 ++-
 .../hadoop/hbase/regionserver/HRegionServer.java   |  13 ++-
 .../hadoop/hbase/security/HBasePolicyProvider.java |   2 +
 .../apache/hadoop/hbase/HBaseTestingUtility.java   |  31 ++++++-
 .../org/apache/hadoop/hbase/MiniClusterRule.java   |   7 ++
 .../org/apache/hadoop/hbase/MiniHBaseCluster.java  |  20 ++--
 .../hadoop/hbase/StartMiniClusterOption.java       |  28 +++++-
 .../hbase/client/AbstractTestRegionLocator.java    |   3 +-
 .../hbase/client/RegionReplicaTestHelper.java      |  24 +++--
 .../hadoop/hbase/client/TestAsyncAdminBase.java    |   5 +-
 .../hbase/client/TestAsyncAdminMasterSwitch.java   |   2 -
 .../client/TestAsyncAdminWithRegionReplicas.java   |   2 +-
 .../hbase/client/TestAsyncMetaRegionLocator.java   |   5 +-
 .../client/TestAsyncTableUseMetaReplicas.java      |   2 +-
 .../hadoop/hbase/client/TestClientTimeouts.java    |  18 +++-
 .../apache/hadoop/hbase/client/TestConnection.java |   3 +
 .../client/TestFromClientSideWithCoprocessor.java  |   1 +
 .../hbase/client/TestMetaRegionLocationCache.java  |   6 +-
 .../hadoop/hbase/client/TestMetaWithReplicas.java  |   5 +-
 .../hbase/client/TestReplicaWithCluster.java       |   7 +-
 .../hadoop/hbase/client/TestReplicasClient.java    |   7 +-
 .../hbase/client/TestZKConnectionRegistry.java     |   2 +-
 .../hadoop/hbase/ipc/TestRpcClientLeaks.java       |  30 +++++-
 .../hadoop/hbase/master/AlwaysStandByHMaster.java  | 101 +++++++++++++++++++++
 .../hbase/master/TestAlwaysStandByHMaster.java     |  67 ++++++++++++++
 .../TestMasterOperationsForRegionReplicas.java     |  16 +++-
 .../hadoop/hbase/master/TestMasterShutdown.java    |  14 ++-
 .../hbase/master/TestMigrateNamespaceTable.java    |   9 +-
 .../assignment/TestRegionMoveAndAbandon.java       |   1 +
 .../hbase/master/assignment/TestRegionSplit.java   |   1 +
 .../hbase/namespace/TestNamespaceAuditor.java      |  23 ++---
 .../regionserver/TestRSKilledWhenInitializing.java |   5 +-
 .../regionserver/TestRegionServerNoMaster.java     |   4 +-
 .../TestSplitTransactionOnCluster.java             |   8 +-
 .../hbase/replication/TestReplicationBase.java     |  27 ++++--
 .../TestReplicationDisableInactivePeer.java        |   4 +-
 .../replication/TestReplicationDroppedTables.java  |  31 +++----
 .../TestReplicationStatusAfterLagging.java         |   2 +-
 ...licationStatusBothNormalAndRecoveryLagging.java |   2 +-
 ...ationStatusSourceStartedTargetStoppedNewOp.java |   2 +-
 ...ationStatusSourceStartedTargetStoppedNoOps.java |   2 +-
 ...atusSourceStartedTargetStoppedWithRecovery.java |   2 +-
 .../replication/TestReplicationSyncUpTool.java     |  18 ++--
 .../replication/TestReplicationSyncUpToolBase.java |  31 ++++++-
 ...estReplicationSyncUpToolWithBulkLoadedData.java |   8 +-
 ...stRegionReplicaReplicationEndpointNoMaster.java |   5 +-
 .../hbase/security/token/SecureTestCluster.java    |   2 +-
 .../token/TestDelegationTokenWithEncryption.java   |   8 +-
 .../token/TestGenerateDelegationToken.java         |   8 +-
 .../security/token/TestTokenAuthentication.java    |   5 +
 58 files changed, 543 insertions(+), 176 deletions(-)

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistryFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistryFactory.java
index 9308443..5688dea 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistryFactory.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistryFactory.java
@@ -36,7 +36,7 @@ final class ConnectionRegistryFactory {
    */
   static ConnectionRegistry getRegistry(Configuration conf) {
     Class<? extends ConnectionRegistry> clazz = conf.getClass(
-        CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY, ZKConnectionRegistry.class,
+        CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY, MasterRegistry.class,
         ConnectionRegistry.class);
     return ReflectionUtils.newInstance(clazz, conf);
   }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
index 5680847..9207538 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
@@ -90,6 +90,7 @@ public class MasterRegistry implements ConnectionRegistry {
     } else {
       finalConf = conf;
     }
+    finalConf.set(MASTER_ADDRS_KEY, conf.get(MASTER_ADDRS_KEY, MASTER_ADDRS_DEFAULT));
     rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE, conf.getLong(HConstants.HBASE_RPC_TIMEOUT_KEY,
         HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
     masterServers = new HashSet<>();
@@ -146,12 +147,13 @@ public class MasterRegistry implements ConnectionRegistry {
       if (rpcResult == null) {
         future.completeExceptionally(
             new MasterRegistryFetchException(masterServers, hrc.getFailed()));
+        return;
       }
       if (!isValidResp.test(rpcResult)) {
         // Rpc returned ok, but result was malformed.
         future.completeExceptionally(new IOException(
             String.format("Invalid result for request %s. Will be retried", debug)));
-
+        return;
       }
       future.complete(transformResult.apply(rpcResult));
     };
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java
index fefeea6..f569e47 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java
@@ -606,7 +606,6 @@ public final class ReplicationPeerConfigUtil {
       compound.addStringMap(peerConfig.getConfiguration());
       return compound;
     }
-
     return otherConf;
   }
 }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SecurityInfo.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SecurityInfo.java
index d96b676..e5b4de2 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SecurityInfo.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SecurityInfo.java
@@ -49,6 +49,8 @@ public class SecurityInfo {
         new SecurityInfo(SecurityConstants.MASTER_KRB_PRINCIPAL, Kind.HBASE_AUTH_TOKEN));
     infos.put(MasterProtos.HbckService.getDescriptor().getName(),
         new SecurityInfo(SecurityConstants.MASTER_KRB_PRINCIPAL, Kind.HBASE_AUTH_TOKEN));
+    infos.put(MasterProtos.ClientMetaService.getDescriptor().getName(),
+        new SecurityInfo(SecurityConstants.MASTER_KRB_PRINCIPAL, Kind.HBASE_AUTH_TOKEN));
     // NOTE: IF ADDING A NEW SERVICE, BE SURE TO UPDATE HBasePolicyProvider ALSO ELSE
     // new Service will not be found when all is Kerberized!!!!
   }
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HBaseConfiguration.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HBaseConfiguration.java
index 0f39e8b..67de5fb 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HBaseConfiguration.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HBaseConfiguration.java
@@ -250,7 +250,7 @@ public class HBaseConfiguration extends Configuration {
    * @return the merged configuration with override properties and cluster key applied
    */
   public static Configuration createClusterConf(Configuration baseConf, String clusterKey,
-                                                String overridePrefix) throws IOException {
+      String overridePrefix) throws IOException {
     Configuration clusterConf = HBaseConfiguration.create(baseConf);
     if (clusterKey != null && !clusterKey.isEmpty()) {
       applyClusterKeyToConf(clusterConf, clusterKey);
@@ -268,14 +268,21 @@ public class HBaseConfiguration extends Configuration {
    * used to communicate with distant clusters
    * @param conf configuration object to configure
    * @param key string that contains the 3 required configuratins
-   * @throws IOException
    */
   private static void applyClusterKeyToConf(Configuration conf, String key)
-      throws IOException{
+      throws IOException {
     ZKConfig.ZKClusterKey zkClusterKey = ZKConfig.transformClusterKey(key);
     conf.set(HConstants.ZOOKEEPER_QUORUM, zkClusterKey.getQuorumString());
     conf.setInt(HConstants.ZOOKEEPER_CLIENT_PORT, zkClusterKey.getClientPort());
     conf.set(HConstants.ZOOKEEPER_ZNODE_PARENT, zkClusterKey.getZnodeParent());
+    // Without the right registry, the above configs are useless. Also, we don't use setClass()
+    // here because the ConnectionRegistry* classes are not resolvable from this module.
+    // This will be broken if ZkConnectionRegistry class gets renamed or moved. Is there a better
+    // way?
+    LOG.info("Overriding client registry implementation to {}",
+        HConstants.ZK_CONNECTION_REGISTRY_CLASS);
+    conf.set(HConstants.CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY,
+        HConstants.ZK_CONNECTION_REGISTRY_CLASS);
   }
 
   /**
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
index a7acc8e..c2354a9 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
@@ -183,6 +183,10 @@ public final class HConstants {
 
   public static final String MASTER_ADDRS_DEFAULT =  "localhost:" + DEFAULT_MASTER_PORT;
 
+  /** Full class name of the Zookeeper based connection registry implementation */
+  public static final String ZK_CONNECTION_REGISTRY_CLASS =
+      "org.apache.hadoop.hbase.client.ZKConnectionRegistry";
+
   /** Configuration to enable hedged reads on master registry **/
   public static final String MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY =
       "hbase.client.master_registry.enable_hedged_reads";
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java
index 1fdd464..b67fffa 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java
@@ -91,7 +91,7 @@ public class LocalHBaseCluster {
    */
   public LocalHBaseCluster(final Configuration conf, final int noRegionServers)
   throws IOException {
-    this(conf, 1, noRegionServers, getMasterImplementation(conf),
+    this(conf, 1, 0, noRegionServers, getMasterImplementation(conf),
         getRegionServerImplementation(conf));
   }
 
@@ -106,7 +106,7 @@ public class LocalHBaseCluster {
   public LocalHBaseCluster(final Configuration conf, final int noMasters,
       final int noRegionServers)
   throws IOException {
-    this(conf, noMasters, noRegionServers, getMasterImplementation(conf),
+    this(conf, noMasters, 0, noRegionServers, getMasterImplementation(conf),
         getRegionServerImplementation(conf));
   }
 
@@ -122,6 +122,12 @@ public class LocalHBaseCluster {
        HMaster.class);
   }
 
+  public LocalHBaseCluster(final Configuration conf, final int noMasters, final int noRegionServers,
+      final Class<? extends HMaster> masterClass,
+      final Class<? extends HRegionServer> regionServerClass) throws IOException {
+    this(conf, noMasters, 0, noRegionServers, masterClass, regionServerClass);
+  }
+
   /**
    * Constructor.
    * @param conf Configuration to use.  Post construction has the master's
@@ -134,9 +140,9 @@ public class LocalHBaseCluster {
    */
   @SuppressWarnings("unchecked")
   public LocalHBaseCluster(final Configuration conf, final int noMasters,
-    final int noRegionServers, final Class<? extends HMaster> masterClass,
-    final Class<? extends HRegionServer> regionServerClass)
-  throws IOException {
+      final int noAlwaysStandByMasters, final int noRegionServers,
+      final Class<? extends HMaster> masterClass,
+      final Class<? extends HRegionServer> regionServerClass) throws IOException {
     this.conf = conf;
 
     // When active, if a port selection is default then we switch to random
@@ -170,24 +176,22 @@ public class LocalHBaseCluster {
     this.masterClass = (Class<? extends HMaster>)
       conf.getClass(HConstants.MASTER_IMPL, masterClass);
     // Start the HMasters.
-    for (int i = 0; i < noMasters; i++) {
+    int i;
+    for (i = 0; i < noMasters; i++) {
       addMaster(new Configuration(conf), i);
     }
-
-    // Populate the master address host ports in the config. This is needed if a master based
-    // registry is configured for client metadata services (HBASE-18095)
-    List<String> masterHostPorts = new ArrayList<>();
-    getMasters().forEach(masterThread ->
-        masterHostPorts.add(masterThread.getMaster().getServerName().getAddress().toString()));
-    conf.set(HConstants.MASTER_ADDRS_KEY, String.join(",", masterHostPorts));
-
+    for (int j = 0; j < noAlwaysStandByMasters; j++) {
+      Configuration c = new Configuration(conf);
+      c.set(HConstants.MASTER_IMPL, "org.apache.hadoop.hbase.master.AlwaysStandByHMaster");
+      addMaster(c, i + j);
+    }
     // Start the HRegionServers.
     this.regionServerClass =
       (Class<? extends HRegionServer>)conf.getClass(HConstants.REGION_SERVER_IMPL,
        regionServerClass);
 
-    for (int i = 0; i < noRegionServers; i++) {
-      addRegionServer(new Configuration(conf), i);
+    for (int j = 0; j < noRegionServers; j++) {
+      addRegionServer(new Configuration(conf), j);
     }
   }
 
@@ -233,8 +237,13 @@ public class LocalHBaseCluster {
     // its Connection instance rather than share (see HBASE_INSTANCES down in
     // the guts of ConnectionManager.
     JVMClusterUtil.MasterThread mt = JVMClusterUtil.createMasterThread(c,
-        (Class<? extends HMaster>) conf.getClass(HConstants.MASTER_IMPL, this.masterClass), index);
+        (Class<? extends HMaster>) c.getClass(HConstants.MASTER_IMPL, this.masterClass), index);
     this.masterThreads.add(mt);
+    // Refresh the master address config.
+    List<String> masterHostPorts = new ArrayList<>();
+    getMasters().forEach(masterThread ->
+        masterHostPorts.add(masterThread.getMaster().getServerName().getAddress().toString()));
+    conf.set(HConstants.MASTER_ADDRS_KEY, String.join(",", masterHostPorts));
     return mt;
   }
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java
index 99cab62..606741b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java
@@ -56,15 +56,15 @@ public class ActiveMasterManager extends ZKListener {
   final AtomicBoolean clusterHasActiveMaster = new AtomicBoolean(false);
   final AtomicBoolean clusterShutDown = new AtomicBoolean(false);
 
-  // This server's information.
-  private final ServerName sn;
-  private int infoPort;
-  private final Server master;
+  // This server's information. Package-private for child implementations.
+  int infoPort;
+  final ServerName sn;
+  final Server master;
 
   // Active master's server name. Invalidated anytime active master changes (based on ZK
   // notifications) and lazily fetched on-demand.
   // ServerName is immutable, so we don't need heavy synchronization around it.
-  private volatile ServerName activeMasterServerName;
+  volatile ServerName activeMasterServerName;
 
   /**
    * @param watcher ZK watcher
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 58f99b5..def82b4 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
@@ -575,7 +575,7 @@ public class HMaster extends HRegionServer implements MasterServices {
       // Some unit tests don't need a cluster, so no zookeeper at all
       if (!conf.getBoolean("hbase.testing.nocluster", false)) {
         this.metaRegionLocationCache = new MetaRegionLocationCache(this.zooKeeper);
-        this.activeMasterManager = new ActiveMasterManager(zooKeeper, this.serverName, this);
+        this.activeMasterManager = createActiveMasterManager(zooKeeper, serverName, this);
       } else {
         this.metaRegionLocationCache = null;
         this.activeMasterManager = null;
@@ -589,6 +589,15 @@ public class HMaster extends HRegionServer implements MasterServices {
     }
   }
 
+  /**
+   * Protected to have custom implementations in tests override the default ActiveMaster
+   * implementation.
+   */
+  protected ActiveMasterManager createActiveMasterManager(
+      ZKWatcher zk, ServerName sn, org.apache.hadoop.hbase.Server server) {
+    return new ActiveMasterManager(zk, sn, server);
+  }
+
   @Override
   protected String getUseThisHostnameInstead(Configuration conf) {
     return conf.get(MASTER_HOSTNAME_KEY);
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 1fbd548..d71578c 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
@@ -789,8 +789,17 @@ public class HRegionServer extends HasThread implements
     return true;
   }
 
-  private Configuration unsetClientZookeeperQuorum() {
+  private Configuration cleanupConfiguration() {
     Configuration conf = this.conf;
+    // We use ZKConnectionRegistry for all the internal communication, primarily for these reasons:
+    // - Decouples RS and master life cycles. RegionServers can continue be up independent of
+    //   masters' availability.
+    // - Configuration management for region servers (cluster internal) is much simpler when adding
+    //   new masters or removing existing masters, since only clients' config needs to be updated.
+    // - We need to retain ZKConnectionRegistry for replication use anyway, so we just extend it for
+    //   other internal connections too.
+    conf.set(HConstants.CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY,
+        HConstants.ZK_CONNECTION_REGISTRY_CLASS);
     if (conf.get(HConstants.CLIENT_ZOOKEEPER_QUORUM) != null) {
       // Use server ZK cluster for server-issued connections, so we clone
       // the conf and unset the client ZK related properties
@@ -824,7 +833,7 @@ public class HRegionServer extends HasThread implements
    */
   protected final synchronized void setupClusterConnection() throws IOException {
     if (asyncClusterConnection == null) {
-      Configuration conf = unsetClientZookeeperQuorum();
+      Configuration conf = cleanupConfiguration();
       InetSocketAddress localAddress = new InetSocketAddress(this.rpcServices.isa.getAddress(), 0);
       User user = userProvider.getCurrent();
       asyncClusterConnection =
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/HBasePolicyProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/HBasePolicyProvider.java
index 72e36a8..b7ab7f0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/HBasePolicyProvider.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/HBasePolicyProvider.java
@@ -40,6 +40,8 @@ public class HBasePolicyProvider extends PolicyProvider {
     new Service("security.client.protocol.acl", AdminService.BlockingInterface.class),
     new Service("security.client.protocol.acl",
       MasterProtos.HbckService.BlockingInterface.class),
+    new Service("security.client.protocol.acl",
+      MasterProtos.ClientMetaService.BlockingInterface.class),
     new Service("security.admin.protocol.acl", MasterService.BlockingInterface.class),
     new Service("security.masterregion.protocol.acl",
       RegionServerStatusService.BlockingInterface.class)
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
index d2f7487..589e4a4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
@@ -1114,8 +1114,9 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
     Configuration c = new Configuration(this.conf);
     TraceUtil.initTracer(c);
     this.hbaseCluster =
-        new MiniHBaseCluster(c, option.getNumMasters(), option.getNumRegionServers(),
-            option.getRsPorts(), option.getMasterClass(), option.getRsClass());
+        new MiniHBaseCluster(c, option.getNumMasters(), option.getNumAlwaysStandByMasters(),
+            option.getNumRegionServers(), option.getRsPorts(), option.getMasterClass(),
+            option.getRsClass());
     // Populate the master address configuration from mini cluster configuration.
     conf.set(HConstants.MASTER_ADDRS_KEY,
         c.get(HConstants.MASTER_ADDRS_KEY, HConstants.MASTER_ADDRS_DEFAULT));
@@ -1231,6 +1232,7 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
     StartMiniClusterOption option =
         StartMiniClusterOption.builder().numRegionServers(servers).rsPorts(ports).build();
     restartHBaseCluster(option);
+    invalidateConnection();
   }
 
   public void restartHBaseCluster(StartMiniClusterOption option)
@@ -1244,8 +1246,9 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
       this.asyncConnection = null;
     }
     this.hbaseCluster =
-        new MiniHBaseCluster(this.conf, option.getNumMasters(), option.getNumRegionServers(),
-            option.getRsPorts(), option.getMasterClass(), option.getRsClass());
+        new MiniHBaseCluster(this.conf, option.getNumMasters(), option.getNumAlwaysStandByMasters(),
+            option.getNumRegionServers(), option.getRsPorts(), option.getMasterClass(),
+            option.getRsClass());
     // Don't leave here till we've done a successful scan of the hbase:meta
     Connection conn = ConnectionFactory.createConnection(this.conf);
     Table t = conn.getTable(TableName.META_TABLE_NAME);
@@ -3020,6 +3023,26 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
   }
 
   /**
+   * Resets the connections so that the next time getConnection() is called, a new connection is
+   * created. This is needed in cases where the entire cluster / all the masters are shutdown and
+   * the connection is not valid anymore.
+   * TODO: There should be a more coherent way of doing this. Unfortunately the way tests are
+   *   written, not all start() stop() calls go through this class. Most tests directly operate on
+   *   the underlying mini/local hbase cluster. That makes it difficult for this wrapper class to
+   *   maintain the connection state automatically. Cleaning this is a much bigger refactor.
+   */
+  public void invalidateConnection() throws IOException {
+    closeConnection();
+    // Update the master addresses if they changed.
+    final String masterConfigBefore = conf.get(HConstants.MASTER_ADDRS_KEY);
+    final String masterConfAfter = getMiniHBaseCluster().conf.get(HConstants.MASTER_ADDRS_KEY);
+    LOG.info("Invalidated connection. Updating master addresses before: {} after: {}",
+        masterConfigBefore, masterConfAfter);
+    conf.set(HConstants.MASTER_ADDRS_KEY,
+        getMiniHBaseCluster().conf.get(HConstants.MASTER_ADDRS_KEY));
+  }
+
+  /**
    * Get a Connection to the cluster. Not thread-safe (This class needs a lot of work to make it
    * thread-safe).
    * @return A Connection that can be shared. Don't close. Will be closed on shutdown of cluster.
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniClusterRule.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniClusterRule.java
index 6ac4838..89fbded4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniClusterRule.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniClusterRule.java
@@ -66,6 +66,13 @@ public class MiniClusterRule extends ExternalResource {
   }
 
   /**
+   * @return the underlying instance of {@link HBaseTestingUtility}
+   */
+  public HBaseTestingUtility getTestingUtility() {
+    return testingUtility;
+  }
+
+  /**
    * Create a {@link AsyncConnection} to the managed {@link MiniHBaseCluster}. It's up to the caller
    * to {@link AsyncConnection#close() close()} the connection when finished.
    */
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java
index 84e6f8f..53c590b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java
@@ -88,7 +88,7 @@ public class MiniHBaseCluster extends HBaseCluster {
          Class<? extends HMaster> masterClass,
          Class<? extends MiniHBaseCluster.MiniHBaseClusterRegionServer> regionserverClass)
       throws IOException, InterruptedException {
-    this(conf, numMasters, numRegionServers, null, masterClass, regionserverClass);
+    this(conf, numMasters, 0, numRegionServers, null, masterClass, regionserverClass);
   }
 
   /**
@@ -99,9 +99,8 @@ public class MiniHBaseCluster extends HBaseCluster {
    * @throws IOException
    * @throws InterruptedException
    */
-  public MiniHBaseCluster(Configuration conf, int numMasters, int numRegionServers,
-         List<Integer> rsPorts,
-         Class<? extends HMaster> masterClass,
+  public MiniHBaseCluster(Configuration conf, int numMasters, int numAlwaysStandByMasters,
+         int numRegionServers, List<Integer> rsPorts, Class<? extends HMaster> masterClass,
          Class<? extends MiniHBaseCluster.MiniHBaseClusterRegionServer> regionserverClass)
       throws IOException, InterruptedException {
     super(conf);
@@ -109,7 +108,8 @@ public class MiniHBaseCluster extends HBaseCluster {
     // Hadoop 2
     CompatibilityFactory.getInstance(MetricsAssertHelper.class).init();
 
-    init(numMasters, numRegionServers, rsPorts, masterClass, regionserverClass);
+    init(numMasters, numAlwaysStandByMasters, numRegionServers, rsPorts, masterClass,
+        regionserverClass);
     this.initialClusterStatus = getClusterMetrics();
   }
 
@@ -225,9 +225,9 @@ public class MiniHBaseCluster extends HBaseCluster {
     }
   }
 
-  private void init(final int nMasterNodes, final int nRegionNodes, List<Integer> rsPorts,
-                 Class<? extends HMaster> masterClass,
-                 Class<? extends MiniHBaseCluster.MiniHBaseClusterRegionServer> regionserverClass)
+  private void init(final int nMasterNodes, final int numAlwaysStandByMasters,
+      final int nRegionNodes, List<Integer> rsPorts, Class<? extends HMaster> masterClass,
+      Class<? extends MiniHBaseCluster.MiniHBaseClusterRegionServer> regionserverClass)
   throws IOException, InterruptedException {
     try {
       if (masterClass == null){
@@ -238,7 +238,7 @@ public class MiniHBaseCluster extends HBaseCluster {
       }
 
       // start up a LocalHBaseCluster
-      hbaseCluster = new LocalHBaseCluster(conf, nMasterNodes, 0,
+      hbaseCluster = new LocalHBaseCluster(conf, nMasterNodes, numAlwaysStandByMasters, 0,
           masterClass, regionserverClass);
 
       // manually add the regionservers as other users
@@ -553,6 +553,8 @@ public class MiniHBaseCluster extends HBaseCluster {
     } catch (InterruptedException ie) {
       throw new IOException("Interrupted adding master to cluster", ie);
     }
+    conf.set(HConstants.MASTER_ADDRS_KEY,
+        hbaseCluster.getConfiguration().get(HConstants.MASTER_ADDRS_KEY));
     return t;
   }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/StartMiniClusterOption.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/StartMiniClusterOption.java
index ad70c95..7a9bd68 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/StartMiniClusterOption.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/StartMiniClusterOption.java
@@ -46,6 +46,14 @@ public final class StartMiniClusterOption {
    * can find the active/primary master with {@link MiniHBaseCluster#getMaster()}.
    */
   private final int numMasters;
+
+  /**
+   * Number of masters that always remain standby. These set of masters never transition to active
+   * even if an active master does not exist. These are needed for testing scenarios where there are
+   * no active masters in the cluster but the cluster connection (backed by master registry) should
+   * still work.
+   */
+  private final int numAlwaysStandByMasters;
   /**
    * The class to use as HMaster, or null for default.
    */
@@ -99,11 +107,12 @@ public final class StartMiniClusterOption {
   /**
    * Private constructor. Use {@link Builder#build()}.
    */
-  private StartMiniClusterOption(int numMasters, Class<? extends HMaster> masterClass,
-      int numRegionServers, List<Integer> rsPorts,
+  private StartMiniClusterOption(int numMasters, int numAlwaysStandByMasters,
+      Class<? extends HMaster> masterClass, int numRegionServers, List<Integer> rsPorts,
       Class<? extends MiniHBaseCluster.MiniHBaseClusterRegionServer> rsClass, int numDataNodes,
       String[] dataNodeHosts, int numZkServers, boolean createRootDir, boolean createWALDir) {
     this.numMasters = numMasters;
+    this.numAlwaysStandByMasters = numAlwaysStandByMasters;
     this.masterClass = masterClass;
     this.numRegionServers = numRegionServers;
     this.rsPorts = rsPorts;
@@ -119,6 +128,10 @@ public final class StartMiniClusterOption {
     return numMasters;
   }
 
+  public int getNumAlwaysStandByMasters() {
+    return numAlwaysStandByMasters;
+  }
+
   public Class<? extends HMaster> getMasterClass() {
     return masterClass;
   }
@@ -179,6 +192,7 @@ public final class StartMiniClusterOption {
    */
   public static final class Builder {
     private int numMasters = 1;
+    private int numAlwaysStandByMasters = 0;
     private Class<? extends HMaster> masterClass = null;
     private int numRegionServers = 1;
     private List<Integer> rsPorts = null;
@@ -196,8 +210,9 @@ public final class StartMiniClusterOption {
       if (dataNodeHosts != null && dataNodeHosts.length != 0) {
         numDataNodes = dataNodeHosts.length;
       }
-      return new StartMiniClusterOption(numMasters, masterClass, numRegionServers, rsPorts, rsClass,
-          numDataNodes, dataNodeHosts, numZkServers, createRootDir, createWALDir);
+      return new StartMiniClusterOption(numMasters,numAlwaysStandByMasters, masterClass,
+          numRegionServers, rsPorts, rsClass, numDataNodes, dataNodeHosts, numZkServers,
+          createRootDir, createWALDir);
     }
 
     public Builder numMasters(int numMasters) {
@@ -205,6 +220,11 @@ public final class StartMiniClusterOption {
       return this;
     }
 
+    public Builder numAlwaysStandByMasters(int numAlwaysStandByMasters) {
+      this.numAlwaysStandByMasters = numAlwaysStandByMasters;
+      return this;
+    }
+
     public Builder masterClass(Class<? extends HMaster> masterClass) {
       this.masterClass = masterClass;
       return this;
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestRegionLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestRegionLocator.java
index f96daf6..89f287b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestRegionLocator.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestRegionLocator.java
@@ -60,8 +60,7 @@ public abstract class AbstractTestRegionLocator {
     UTIL.waitTableAvailable(TABLE_NAME);
     try (ConnectionRegistry registry =
              ConnectionRegistryFactory.getRegistry(UTIL.getConfiguration())) {
-      RegionReplicaTestHelper.waitUntilAllMetaReplicasHavingRegionLocation(UTIL.getConfiguration(),
-        registry, REGION_REPLICATION);
+      RegionReplicaTestHelper.waitUntilAllMetaReplicasAreReady(UTIL, registry);
     }
     UTIL.getAdmin().balancerSwitch(false, true);
   }
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
index 73924a3..8e562bd 100644
--- 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
@@ -20,13 +20,13 @@ package org.apache.hadoop.hbase.client;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.fail;
-
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Optional;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.NotServingRegionException;
 import org.apache.hadoop.hbase.RegionLocations;
@@ -43,24 +43,32 @@ public final class RegionReplicaTestHelper {
   }
 
   // waits for all replicas to have region location
-  static void waitUntilAllMetaReplicasHavingRegionLocation(Configuration conf,
-      ConnectionRegistry registry, int regionReplication) throws IOException {
+  static void waitUntilAllMetaReplicasAreReady(HBaseTestingUtility util,
+      ConnectionRegistry registry) {
+    Configuration conf = util.getConfiguration();
+    int regionReplicaCount = util.getConfiguration().getInt(HConstants.META_REPLICAS_NUM,
+        HConstants.DEFAULT_META_REPLICA_NUM);
     Waiter.waitFor(conf, conf.getLong("hbase.client.sync.wait.timeout.msec", 60000), 200, true,
       new ExplainingPredicate<IOException>() {
         @Override
-        public String explainFailure() throws IOException {
+        public String explainFailure() {
           return "Not all meta replicas get assigned";
         }
 
         @Override
-        public boolean evaluate() throws IOException {
+        public boolean evaluate() {
           try {
             RegionLocations locs = registry.getMetaRegionLocations().get();
-            if (locs.size() < regionReplication) {
+            if (locs.size() < regionReplicaCount) {
               return false;
             }
-            for (int i = 0; i < regionReplication; i++) {
-              if (locs.getRegionLocation(i) == null) {
+            for (int i = 0; i < regionReplicaCount; i++) {
+              HRegionLocation loc = locs.getRegionLocation(i);
+              // Wait until the replica is served by a region server. There could be delay between
+              // the replica being available to the connection and region server opening it.
+              Optional<ServerName> rsCarryingReplica =
+                  getRSCarryingReplica(util, loc.getRegion().getTable(), i);
+              if (!rsCarryingReplica.isPresent()) {
                 return false;
               }
             }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminBase.java
index a02ee90..6090d64 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminBase.java
@@ -29,6 +29,7 @@ import java.util.regex.Pattern;
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.StartMiniClusterOption;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.After;
@@ -83,7 +84,9 @@ public abstract class TestAsyncAdminBase {
     TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT, 120000);
     TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 2);
     TEST_UTIL.getConfiguration().setInt(START_LOG_ERRORS_AFTER_COUNT_KEY, 0);
-    TEST_UTIL.startMiniCluster(2);
+    StartMiniClusterOption option = StartMiniClusterOption.builder().numRegionServers(2).
+        numMasters(2).build();
+    TEST_UTIL.startMiniCluster(option);
     ASYNC_CONN = ConnectionFactory.createAsyncConnection(TEST_UTIL.getConfiguration()).get();
   }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminMasterSwitch.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminMasterSwitch.java
index 159dce9..ce91e54 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminMasterSwitch.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminMasterSwitch.java
@@ -48,8 +48,6 @@ public class TestAsyncAdminMasterSwitch extends TestAsyncAdminBase {
     assertEquals(TEST_UTIL.getHBaseCluster().getRegionServerThreads().size(),
       admin.getClusterMetrics(EnumSet.of(ClusterMetrics.Option.SERVERS_NAME)).join()
         .getServersName().size());
-    // stop the old master, and start a new one
-    TEST_UTIL.getMiniHBaseCluster().startMaster();
     TEST_UTIL.getMiniHBaseCluster().stopMaster(0).join();
     assertTrue(TEST_UTIL.getMiniHBaseCluster().waitForActiveAndReadyMaster(30000));
     // make sure that we could still call master
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminWithRegionReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminWithRegionReplicas.java
index 6d7d368..50111f7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminWithRegionReplicas.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminWithRegionReplicas.java
@@ -56,7 +56,7 @@ public class TestAsyncAdminWithRegionReplicas extends TestAsyncAdminBase {
     try (ConnectionRegistry registry =
              ConnectionRegistryFactory.getRegistry(TEST_UTIL.getConfiguration())) {
       RegionReplicaTestHelper
-        .waitUntilAllMetaReplicasHavingRegionLocation(TEST_UTIL.getConfiguration(), registry, 3);
+        .waitUntilAllMetaReplicasAreReady(TEST_UTIL, registry);
     }
   }
 
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 609a129..34683e1 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
@@ -51,12 +51,11 @@ public class TestAsyncMetaRegionLocator {
 
   @BeforeClass
   public static void setUp() throws Exception {
-    TEST_UTIL.getConfiguration().set(BaseLoadBalancer.TABLES_ON_MASTER, "none");
+    TEST_UTIL.getConfiguration().setBoolean(BaseLoadBalancer.TABLES_ON_MASTER, false);
     TEST_UTIL.getConfiguration().setInt(HConstants.META_REPLICAS_NUM, 3);
     TEST_UTIL.startMiniCluster(3);
     REGISTRY = ConnectionRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
-    RegionReplicaTestHelper
-      .waitUntilAllMetaReplicasHavingRegionLocation(TEST_UTIL.getConfiguration(), REGISTRY, 3);
+    RegionReplicaTestHelper.waitUntilAllMetaReplicasAreReady(TEST_UTIL, REGISTRY);
     TEST_UTIL.getAdmin().balancerSwitch(false, true);
     LOCATOR = new AsyncMetaRegionLocator(REGISTRY);
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableUseMetaReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableUseMetaReplicas.java
index ed6c66f..6eadba9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableUseMetaReplicas.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableUseMetaReplicas.java
@@ -93,7 +93,7 @@ public class TestAsyncTableUseMetaReplicas {
       FailPrimaryMetaScanCp.class.getName());
     UTIL.startMiniCluster(3);
     try (ConnectionRegistry registry = ConnectionRegistryFactory.getRegistry(conf)) {
-      RegionReplicaTestHelper.waitUntilAllMetaReplicasHavingRegionLocation(conf, registry, 3);
+      RegionReplicaTestHelper.waitUntilAllMetaReplicasAreReady(UTIL, registry);
     }
     try (Table table = UTIL.createTable(TABLE_NAME, FAMILY)) {
       table.put(new Put(ROW).addColumn(FAMILY, QUALIFIER, VALUE));
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java
index bd1f7cc..a7991c7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java
@@ -19,11 +19,11 @@ package org.apache.hadoop.hbase.client;
 
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
-
 import java.net.InetSocketAddress;
 import java.net.SocketAddress;
 import java.net.SocketTimeoutException;
 import java.net.UnknownHostException;
+import java.util.Set;
 import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.hadoop.conf.Configuration;
@@ -31,8 +31,8 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.MasterNotRunningException;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.exceptions.MasterRegistryFetchException;
 import org.apache.hadoop.hbase.ipc.AbstractRpcClient;
 import org.apache.hadoop.hbase.ipc.BlockingRpcClient;
 import org.apache.hadoop.hbase.ipc.HBaseRpcController;
@@ -45,7 +45,7 @@ import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-
+import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
 import org.apache.hbase.thirdparty.com.google.protobuf.BlockingRpcChannel;
 import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.MethodDescriptor;
 import org.apache.hbase.thirdparty.com.google.protobuf.Message;
@@ -100,9 +100,9 @@ public class TestClientTimeouts {
           connection = ConnectionFactory.createConnection(conf);
           admin = connection.getAdmin();
           admin.balancerSwitch(false, false);
-        } catch (MasterNotRunningException ex) {
+        } catch (MasterRegistryFetchException ex) {
           // Since we are randomly throwing SocketTimeoutExceptions, it is possible to get
-          // a MasterNotRunningException. It's a bug if we get other exceptions.
+          // a MasterRegistryFetchException. It's a bug if we get other exceptions.
           lastFailed = true;
         } finally {
           if (admin != null) {
@@ -146,6 +146,14 @@ public class TestClientTimeouts {
         throws UnknownHostException {
       return new RandomTimeoutRpcChannel(this, sn, ticket, rpcTimeout);
     }
+
+    @Override
+    public RpcChannel createHedgedRpcChannel(Set<ServerName> sns, User user, int rpcTimeout)
+        throws UnknownHostException {
+      Preconditions.checkArgument(sns != null && sns.size() == 1);
+      return new RandomTimeoutRpcChannel(this, (ServerName)sns.toArray()[0], user, rpcTimeout);
+    }
+
   }
 
   private static AtomicInteger invokations = new AtomicInteger();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestConnection.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestConnection.java
index 010b883..b92a4d7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestConnection.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestConnection.java
@@ -315,6 +315,8 @@ public class TestConnection {
     assertTrue(c1.getConfiguration() == c2.getConfiguration());
   }
 
+  /*
+  ====> With MasterRegistry, connections cannot outlast the masters' lifetime.
   @Test
   public void testConnectionRideOverClusterRestart() throws IOException, InterruptedException {
     Configuration config = new Configuration(TEST_UTIL.getConfiguration());
@@ -337,6 +339,7 @@ public class TestConnection {
     table.close();
     connection.close();
   }
+   */
 
   @Test
   public void testLocateRegionsWithRegionReplicas() throws IOException {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideWithCoprocessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideWithCoprocessor.java
index 8845f9a..d78976e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideWithCoprocessor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideWithCoprocessor.java
@@ -43,6 +43,7 @@ public class TestFromClientSideWithCoprocessor extends TestFromClientSide {
   @Parameterized.Parameters
   public static Collection parameters() {
     return Arrays.asList(new Object[][] {
+        { MasterRegistry.class, 1},
         { ZKConnectionRegistry.class, 1}
     });
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java
index 1205b05..abaf092 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java
@@ -60,8 +60,7 @@ public class TestMetaRegionLocationCache {
     TEST_UTIL.getConfiguration().setInt(HConstants.META_REPLICAS_NUM, 3);
     TEST_UTIL.startMiniCluster(3);
     REGISTRY = ConnectionRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
-    RegionReplicaTestHelper.waitUntilAllMetaReplicasHavingRegionLocation(
-        TEST_UTIL.getConfiguration(), REGISTRY, 3);
+    RegionReplicaTestHelper.waitUntilAllMetaReplicasAreReady(TEST_UTIL, REGISTRY);
     TEST_UTIL.getAdmin().balancerSwitch(false, true);
   }
 
@@ -123,8 +122,7 @@ public class TestMetaRegionLocationCache {
     for (HRegionLocation location: currentMetaLocs) {
       RegionReplicaTestHelper.moveRegion(TEST_UTIL, location);
     }
-    RegionReplicaTestHelper.waitUntilAllMetaReplicasHavingRegionLocation(
-        TEST_UTIL.getConfiguration(), REGISTRY, 3);
+    RegionReplicaTestHelper.waitUntilAllMetaReplicasAreReady(TEST_UTIL, REGISTRY);
     for (JVMClusterUtil.MasterThread masterThread:
         TEST_UTIL.getMiniHBaseCluster().getMasterThreads()) {
       verifyCachedMetaLocations(masterThread.getMaster());
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
index 55fc289..809aee8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
@@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.StartMiniClusterOption;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
@@ -85,7 +86,9 @@ public class TestMetaWithReplicas {
     TEST_UTIL.getConfiguration().setInt(HConstants.META_REPLICAS_NUM, 3);
     TEST_UTIL.getConfiguration().setInt(
         StorefileRefresherChore.REGIONSERVER_STOREFILE_REFRESH_PERIOD, 1000);
-    TEST_UTIL.startMiniCluster(REGIONSERVERS_COUNT);
+    StartMiniClusterOption option = StartMiniClusterOption.builder().
+        numAlwaysStandByMasters(1).numMasters(1).numRegionServers(REGIONSERVERS_COUNT).build();
+    TEST_UTIL.startMiniCluster(option);
     AssignmentManager am = TEST_UTIL.getMiniHBaseCluster().getMaster().getAssignmentManager();
     Set<ServerName> sns = new HashSet<ServerName>();
     ServerName hbaseMetaServerName =
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
index acf250c..149f8aa 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
@@ -588,11 +588,12 @@ public class TestReplicaWithCluster {
   }
 
   @Test
-  public void testReplicaGetWithRpcClientImpl() throws IOException {
+  public void testReplicaGetWithAsyncRpcClientImpl() throws IOException {
     HTU.getConfiguration().setBoolean("hbase.ipc.client.specificThreadForWriting", true);
-    HTU.getConfiguration().set("hbase.rpc.client.impl", "org.apache.hadoop.hbase.ipc.RpcClientImpl");
+    HTU.getConfiguration().set(
+        "hbase.rpc.client.impl", "org.apache.hadoop.hbase.ipc.AsyncRpcClient");
     // Create table then get the single region for our new table.
-    HTableDescriptor hdt = HTU.createTableDescriptor("testReplicaGetWithRpcClientImpl");
+    HTableDescriptor hdt = HTU.createTableDescriptor("testReplicaGetWithAsyncRpcClientImpl");
     hdt.setRegionReplication(NB_SERVERS);
     hdt.addCoprocessor(SlowMeCopro.class.getName());
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java
index cacc39a..3e507d7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java
@@ -21,7 +21,6 @@ import com.codahale.metrics.Counter;
 import java.io.IOException;
 import java.util.List;
 import java.util.Optional;
-import java.util.Random;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
@@ -37,6 +36,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.NotServingRegionException;
+import org.apache.hadoop.hbase.StartMiniClusterOption;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
@@ -104,7 +104,6 @@ public class TestReplicasClient {
         new AtomicReference<>(new CountDownLatch(0));
     private static final AtomicReference<CountDownLatch> secondaryCdl =
         new AtomicReference<>(new CountDownLatch(0));
-    Random r = new Random();
     public SlowMeCopro() {
     }
 
@@ -192,7 +191,9 @@ public class TestReplicasClient {
         StorefileRefresherChore.REGIONSERVER_STOREFILE_REFRESH_PERIOD, REFRESH_PERIOD);
     HTU.getConfiguration().setBoolean("hbase.client.log.scanner.activity", true);
     HTU.getConfiguration().setBoolean(MetricsConnection.CLIENT_SIDE_METRICS_ENABLED_KEY, true);
-    HTU.startMiniCluster(NB_SERVERS);
+    StartMiniClusterOption option = StartMiniClusterOption.builder().numRegionServers(1).
+        numAlwaysStandByMasters(1).numMasters(1).build();
+    HTU.startMiniCluster(option);
 
     // Create table then get the single region for our new table.
     HTableDescriptor hdt = HTU.createTableDescriptor(TestReplicasClient.class.getSimpleName());
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKConnectionRegistry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKConnectionRegistry.java
index cb45ee5..d8a228e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKConnectionRegistry.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKConnectionRegistry.java
@@ -86,7 +86,7 @@ public class TestZKConnectionRegistry {
     assertEquals(TEST_UTIL.getHBaseCluster().getMaster().getServerName(),
       REGISTRY.getActiveMaster().get());
     RegionReplicaTestHelper
-      .waitUntilAllMetaReplicasHavingRegionLocation(TEST_UTIL.getConfiguration(), REGISTRY, 3);
+      .waitUntilAllMetaReplicasAreReady(TEST_UTIL, REGISTRY);
     RegionLocations locs = REGISTRY.getMetaRegionLocations().get();
     assertEquals(3, locs.getRegionLocations().length);
     IntStream.range(0, 3).forEach(i -> {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcClientLeaks.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcClientLeaks.java
index 7cce8e8..27c1235 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcClientLeaks.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcClientLeaks.java
@@ -24,12 +24,15 @@ import static org.junit.Assert.fail;
 import java.io.IOException;
 import java.net.Socket;
 import java.net.SocketAddress;
+import java.net.UnknownHostException;
+import java.util.Set;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.LinkedBlockingQueue;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
@@ -37,6 +40,7 @@ import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.MetricsConnection;
 import org.apache.hadoop.hbase.client.RetriesExhaustedException;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.AfterClass;
@@ -48,6 +52,8 @@ import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 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.RpcChannel;
 
 @Category(MediumTests.class)
 public class TestRpcClientLeaks {
@@ -63,6 +69,9 @@ public class TestRpcClientLeaks {
 
   public static class MyRpcClientImpl extends BlockingRpcClient {
 
+    // Exceptions thrown only when this is set to false.
+    private static boolean throwException = false;
+
     public MyRpcClientImpl(Configuration conf) {
       super(conf);
     }
@@ -78,12 +87,26 @@ public class TestRpcClientLeaks {
         @Override
         protected synchronized void setupConnection() throws IOException {
           super.setupConnection();
-          SAVED_SOCKETS.add(socket);
-          throw new IOException(
-            "Sample exception for verifying socket closure in case of exceptions.");
+          if (throwException) {
+            SAVED_SOCKETS.add(socket);
+            throw new IOException(
+                "Sample exception for verifying socket closure in case of exceptions.");
+          }
         }
       };
     }
+
+    // To keep the registry paths happy.
+    @Override
+    public RpcChannel createHedgedRpcChannel(Set<ServerName> sns, User user, int rpcTimeout)
+        throws UnknownHostException {
+      Preconditions.checkState(sns != null && sns.size() == 1);
+      return super.createRpcChannel((ServerName)sns.toArray()[0], user, rpcTimeout);
+    }
+
+    public static void enableThrowExceptions() {
+      throwException = true;
+    }
   }
 
   private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
@@ -110,6 +133,7 @@ public class TestRpcClientLeaks {
     conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 2);
     try (Connection connection = ConnectionFactory.createConnection(conf);
       Table table = connection.getTable(TableName.valueOf(name.getMethodName()))) {
+      MyRpcClientImpl.enableThrowExceptions();
       table.get(new Get(Bytes.toBytes("asd")));
       fail("Should fail because the injected error");
     } catch (RetriesExhaustedException e) {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/AlwaysStandByHMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/AlwaysStandByHMaster.java
new file mode 100644
index 0000000..41a008a
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/AlwaysStandByHMaster.java
@@ -0,0 +1,101 @@
+/*
+ * 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.master;
+
+import java.io.IOException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Server;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.monitoring.MonitoredTask;
+import org.apache.hadoop.hbase.util.Threads;
+import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An implementation of HMaster that always runs as a stand by and never transitions to active.
+ */
+public class AlwaysStandByHMaster extends HMaster {
+  /**
+   * An implementation of ActiveMasterManager that never transitions it's master to active state. It
+   * always remains as a stand by master. With the master registry implementation (HBASE-18095) it
+   * is expected to have at least one active / standby master always running at any point in time
+   * since they serve as the gateway for client connections.
+   *
+   * With this implementation, tests can simulate the scenario of not having an active master yet
+   * the client connections to the cluster succeed.
+   */
+  private static class AlwaysStandByMasterManager extends ActiveMasterManager {
+    private static final Logger LOG =
+        LoggerFactory.getLogger(AlwaysStandByMasterManager.class);
+
+    AlwaysStandByMasterManager(ZKWatcher watcher, ServerName sn, Server master) {
+      super(watcher, sn, master);
+    }
+
+    /**
+     * An implementation that never transitions to an active master.
+     */
+    boolean blockUntilBecomingActiveMaster(int checkInterval, MonitoredTask startupStatus) {
+      while (!(master.isAborted() || master.isStopped())) {
+        startupStatus.setStatus("Forever looping to stay as a standby master.");
+        try {
+          activeMasterServerName = null;
+          try {
+            if (MasterAddressTracker.getMasterAddress(watcher) != null) {
+              clusterHasActiveMaster.set(true);
+            }
+            Threads.sleepWithoutInterrupt(100);
+          } catch (IOException e) {
+            // pass, we will get notified when some other active master creates the znode.
+          }
+        } catch (KeeperException e) {
+          master.abort("Received an unexpected KeeperException, aborting", e);
+          return false;
+        }
+        synchronized (this.clusterHasActiveMaster) {
+          while (clusterHasActiveMaster.get() && !master.isStopped()) {
+            try {
+              clusterHasActiveMaster.wait(checkInterval);
+            } catch (InterruptedException e) {
+              // We expect to be interrupted when a master dies,
+              //  will fall out if so
+              LOG.debug("Interrupted waiting for master to die", e);
+            }
+          }
+          if (clusterShutDown.get()) {
+            this.master.stop(
+                "Cluster went down before this master became active");
+          }
+        }
+      }
+      return false;
+    }
+  }
+
+  public AlwaysStandByHMaster(Configuration conf) throws IOException {
+    super(conf);
+  }
+
+  protected ActiveMasterManager createActiveMasterManager(
+      ZKWatcher zk, ServerName sn, org.apache.hadoop.hbase.Server server) {
+    return new AlwaysStandByMasterManager(zk, sn, server);
+  }
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAlwaysStandByHMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAlwaysStandByHMaster.java
new file mode 100644
index 0000000..a49ae50
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAlwaysStandByHMaster.java
@@ -0,0 +1,67 @@
+/*
+ * 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.master;
+
+import static junit.framework.TestCase.assertTrue;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.MiniClusterRule;
+import org.apache.hadoop.hbase.StartMiniClusterOption;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({MediumTests.class, MasterTests.class})
+public class TestAlwaysStandByHMaster {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestAlwaysStandByHMaster.class);
+
+  private static final StartMiniClusterOption OPTION = StartMiniClusterOption.builder().
+      numAlwaysStandByMasters(1).numMasters(1).numRegionServers(1).build();
+
+  @ClassRule
+  public static final MiniClusterRule miniClusterRule = new MiniClusterRule(OPTION);
+
+  /**
+   * Tests that the AlwaysStandByHMaster does not transition to active state even if no active
+   * master exists.
+   */
+  @Test  public void testAlwaysStandBy() throws Exception {
+    HBaseTestingUtility testUtil = miniClusterRule.getTestingUtility();
+    // Make sure there is an active master.
+    assertNotNull(testUtil.getMiniHBaseCluster().getMaster());
+    assertEquals(2, testUtil.getMiniHBaseCluster().getMasterThreads().size());
+    // Kill the only active master.
+    testUtil.getMiniHBaseCluster().stopMaster(0).join();
+    // Wait for 5s to make sure the always standby doesn't transition to active state.
+    assertFalse(testUtil.getMiniHBaseCluster().waitForActiveAndReadyMaster(5000));
+    // Add a new master.
+    HMaster newActive = testUtil.getMiniHBaseCluster().startMaster().getMaster();
+    assertTrue(testUtil.getMiniHBaseCluster().waitForActiveAndReadyMaster(5000));
+    // Newly added master should be the active.
+    assertEquals(newActive.getServerName(),
+        testUtil.getMiniHBaseCluster().getMaster().getServerName());
+  }
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java
index 5fdecae..952cc0c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java
@@ -34,6 +34,7 @@ import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.ClusterMetrics.Option;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
@@ -86,6 +87,8 @@ public class TestMasterOperationsForRegionReplicas {
   private static Connection CONNECTION = null;
   private static Admin ADMIN;
   private static int numSlaves = 2;
+  private final static StartMiniClusterOption option = StartMiniClusterOption.builder().
+      numRegionServers(numSlaves).numMasters(1).numAlwaysStandByMasters(1).build();
   private static Configuration conf;
 
   @Rule
@@ -95,16 +98,21 @@ public class TestMasterOperationsForRegionReplicas {
   public static void setupBeforeClass() throws Exception {
     conf = TEST_UTIL.getConfiguration();
     conf.setBoolean("hbase.tests.use.shortcircuit.reads", false);
-    TEST_UTIL.startMiniCluster(numSlaves);
+    TEST_UTIL.startMiniCluster(option);
     TEST_UTIL.getAdmin().balancerSwitch(false, true);
-    CONNECTION = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration());
-    ADMIN = CONNECTION.getAdmin();
+    resetConnections();
     while (ADMIN.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)).getLiveServerMetrics()
       .size() < numSlaves) {
       Thread.sleep(100);
     }
   }
 
+  private static void resetConnections() throws IOException {
+    IOUtils.closeQuietly(ADMIN, CONNECTION);
+    CONNECTION = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration());
+    ADMIN = CONNECTION.getAdmin();
+  }
+
   @AfterClass
   public static void tearDownAfterClass() throws Exception {
     Closeables.close(ADMIN, true);
@@ -199,6 +207,7 @@ public class TestMasterOperationsForRegionReplicas {
       TEST_UTIL.startMiniHBaseCluster(option);
       TEST_UTIL.waitUntilAllRegionsAssigned(tableName);
       TEST_UTIL.waitUntilNoRegionsInTransition();
+      resetConnections();
       validateFromSnapshotFromMeta(TEST_UTIL, tableName, numRegions, numReplica,
         ADMIN.getConnection());
 
@@ -208,6 +217,7 @@ public class TestMasterOperationsForRegionReplicas {
       TEST_UTIL.startMiniHBaseCluster();
       TEST_UTIL.waitUntilAllRegionsAssigned(tableName);
       TEST_UTIL.waitUntilNoRegionsInTransition();
+      resetConnections();
       validateSingleRegionServerAssignment(ADMIN.getConnection(), numRegions, numReplica);
       for (int i = 1; i < numSlaves; i++) { // restore the cluster
         TEST_UTIL.getMiniHBaseCluster().startRegionServer();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterShutdown.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterShutdown.java
index 5ce7886..6bbf2c0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterShutdown.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterShutdown.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.LocalHBaseCluster;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.StartMiniClusterOption;
+import org.apache.hadoop.hbase.Waiter;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
@@ -132,12 +133,19 @@ public class TestMasterShutdown {
     util.startMiniDFSCluster(3);
     util.startMiniZKCluster();
     util.createRootDir();
-    final LocalHBaseCluster cluster =
-        new LocalHBaseCluster(conf, NUM_MASTERS, NUM_RS, HMaster.class,
-            MiniHBaseCluster.MiniHBaseClusterRegionServer.class);
+    final LocalHBaseCluster cluster = new LocalHBaseCluster(conf, NUM_MASTERS, NUM_RS,
+        HMaster.class, MiniHBaseCluster.MiniHBaseClusterRegionServer.class);
     final int MASTER_INDEX = 0;
     final MasterThread master = cluster.getMasters().get(MASTER_INDEX);
     master.start();
+    // Switching to master registry exposed a race in the master bootstrap that can result in a
+    // lost shutdown command (HBASE-8422). The race is essentially because the server manager in
+    // HMaster is not initialized by the time shutdown() RPC (below) is made to
+    // the master. The reason it was not happening earlier is because the connection creation with
+    // ZK registry is so slow that by then the server manager is init'ed thus masking the problem.
+    // For now, I'm putting a wait() here to workaround the issue, I think the fix for it is a
+    // little delicate and needs to be done separately.
+    Waiter.waitFor(conf, 5000, () -> master.getMaster().getServerManager() != null);
     LOG.info("Called master start on " + master.getName());
     Thread shutdownThread = new Thread("Shutdown-Thread") {
       @Override
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMigrateNamespaceTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMigrateNamespaceTable.java
index d114317..b4d378f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMigrateNamespaceTable.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMigrateNamespaceTable.java
@@ -24,6 +24,7 @@ import java.io.IOException;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.StartMiniClusterOption;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Table;
@@ -55,11 +56,13 @@ public class TestMigrateNamespaceTable {
 
   @BeforeClass
   public static void setUp() throws Exception {
-    UTIL.startMiniCluster(1);
+    StartMiniClusterOption option = StartMiniClusterOption.builder().numMasters(1).
+        numAlwaysStandByMasters(1).numRegionServers(1).build();
+    UTIL.startMiniCluster(option);
   }
 
   @AfterClass
-  public static void tearDow() throws Exception {
+  public static void tearDown() throws Exception {
     UTIL.shutdownMiniCluster();
   }
 
@@ -82,6 +85,7 @@ public class TestMigrateNamespaceTable {
     masterThread.getMaster().stop("For testing");
     masterThread.join();
     UTIL.getMiniHBaseCluster().startMaster();
+
     // 5 + default and system('hbase')
     assertEquals(7, UTIL.getAdmin().listNamespaceDescriptors().length);
     for (int i = 0; i < 5; i++) {
@@ -95,7 +99,6 @@ public class TestMigrateNamespaceTable {
     masterThread = UTIL.getMiniHBaseCluster().getMasterThread();
     masterThread.getMaster().stop("For testing");
     masterThread.join();
-
     UTIL.getMiniHBaseCluster().startMaster();
 
     // make sure that we could still restart the cluster after disabling the namespace table.
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionMoveAndAbandon.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionMoveAndAbandon.java
index bb95a6f..61e55fa 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionMoveAndAbandon.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionMoveAndAbandon.java
@@ -116,6 +116,7 @@ public class TestRegionMoveAndAbandon {
     // Start up everything again
     LOG.info("Starting cluster");
     UTIL.getMiniHBaseCluster().startMaster();
+    UTIL.invalidateConnection();
     UTIL.ensureSomeRegionServersAvailable(2);
 
     UTIL.waitFor(30_000, new Waiter.Predicate<Exception>() {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionSplit.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionSplit.java
index f723af8..5156ce9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionSplit.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionSplit.java
@@ -141,6 +141,7 @@ public class TestRegionSplit {
     JVMClusterUtil.MasterThread t = UTIL.getHBaseCluster().startMaster();
     Thread.sleep(500);
 
+    UTIL.invalidateConnection();
     // enable table
     UTIL.getAdmin().enableTable(tableName);
     Thread.sleep(500);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
index c961ad3..be95714 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
@@ -32,7 +32,6 @@ import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
-import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -45,6 +44,7 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.StartMiniClusterOption;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Waiter;
 import org.apache.hadoop.hbase.client.Admin;
@@ -117,7 +117,8 @@ public class TestNamespaceAuditor {
     conf.setBoolean(QuotaUtil.QUOTA_CONF_KEY, true);
     conf.setClass("hbase.coprocessor.regionserver.classes", CPRegionServerObserver.class,
       RegionServerObserver.class);
-    UTIL.startMiniCluster();
+    StartMiniClusterOption option = StartMiniClusterOption.builder().numMasters(2).build();
+    UTIL.startMiniCluster(option);
     waitForQuotaInitialize(UTIL);
     ADMIN = UTIL.getAdmin();
   }
@@ -483,17 +484,6 @@ public class TestNamespaceAuditor {
     return getQuotaManager().getState(namespace);
   }
 
-  byte[] getSplitKey(byte[] startKey, byte[] endKey) {
-    String skey = Bytes.toString(startKey);
-    int key;
-    if (StringUtils.isBlank(skey)) {
-      key = Integer.parseInt(Bytes.toString(endKey))/2 ;
-    } else {
-      key = (int) (Integer.parseInt(skey) * 1.5);
-    }
-    return Bytes.toBytes("" + key);
-  }
-
   public static class CustomObserver implements RegionCoprocessor, RegionObserver {
     volatile CountDownLatch postCompact;
 
@@ -546,11 +536,11 @@ public class TestNamespaceAuditor {
     UTIL.waitFor(1000, new Waiter.Predicate<Exception>() {
       @Override
       public boolean evaluate() throws Exception {
-       return (getNamespaceState(nsp1).getTables().size() == 2);
+        return (getNamespaceState(nsp1).getTables().size() == 2);
       }
     });
     NamespaceTableAndRegionInfo before = getNamespaceState(nsp1);
-    restartMaster();
+    killActiveMaster();
     NamespaceTableAndRegionInfo after = getNamespaceState(nsp1);
     assertEquals("Expected: " + before.getTables() + " Found: " + after.getTables(), before
         .getTables().size(), after.getTables().size());
@@ -570,10 +560,9 @@ public class TestNamespaceAuditor {
     });
   }
 
-  private void restartMaster() throws Exception {
+  private void killActiveMaster() throws Exception {
     UTIL.getHBaseCluster().getMaster(0).stop("Stopping to start again");
     UTIL.getHBaseCluster().waitOnMaster(0);
-    UTIL.getHBaseCluster().startMaster();
     waitForQuotaInitialize(UTIL);
   }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSKilledWhenInitializing.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSKilledWhenInitializing.java
index 16ce590..647feec 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSKilledWhenInitializing.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSKilledWhenInitializing.java
@@ -95,9 +95,8 @@ public class TestRSKilledWhenInitializing {
     TEST_UTIL.startMiniDFSCluster(3);
     TEST_UTIL.startMiniZKCluster();
     TEST_UTIL.createRootDir();
-    final LocalHBaseCluster cluster =
-        new LocalHBaseCluster(conf, NUM_MASTERS, NUM_RS, HMaster.class,
-            RegisterAndDieRegionServer.class);
+    final LocalHBaseCluster cluster = new LocalHBaseCluster(conf, NUM_MASTERS, NUM_RS,
+        HMaster.class, RegisterAndDieRegionServer.class);
     final MasterThread master = startMaster(cluster.getMasters().get(0));
     try {
       // Master is up waiting on RegionServers to check in. Now start RegionServers.
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java
index f0c7f2f..4882cad 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java
@@ -146,7 +146,9 @@ public class TestRegionServerNoMaster {
   @AfterClass
   public static void afterClass() throws Exception {
     HRegionServer.TEST_SKIP_REPORTING_TRANSITION = false;
-    table.close();
+    if (table != null) {
+      table.close();
+    }
     HTU.shutdownMiniCluster();
   }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
index 65bd4f5..d9f5e83 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
@@ -35,6 +35,7 @@ import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -142,7 +143,8 @@ public class TestSplitTransactionOnCluster {
   @BeforeClass public static void before() throws Exception {
     TESTING_UTIL.getConfiguration().setInt(HConstants.HBASE_BALANCER_PERIOD, 60000);
     StartMiniClusterOption option = StartMiniClusterOption.builder()
-        .masterClass(MyMaster.class).numRegionServers(NB_SERVERS).numDataNodes(NB_SERVERS).build();
+        .masterClass(MyMaster.class).numRegionServers(NB_SERVERS).
+            numDataNodes(NB_SERVERS).build();
     TESTING_UTIL.startMiniCluster(option);
   }
 
@@ -814,6 +816,10 @@ public class TestSplitTransactionOnCluster {
     cluster.waitOnMaster(0);
     HMaster master = cluster.startMaster().getMaster();
     cluster.waitForActiveAndReadyMaster();
+    // reset the connections
+    IOUtils.closeQuietly(admin);
+    TESTING_UTIL.invalidateConnection();
+    admin = TESTING_UTIL.getAdmin();
     return master;
   }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java
index 94150b8..6065454 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java
@@ -24,6 +24,7 @@ import static org.junit.Assert.fail;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
+import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -75,8 +76,8 @@ public class TestReplicationBase {
 
   protected static final HBaseTestingUtility UTIL1 = new HBaseTestingUtility();
   protected static final HBaseTestingUtility UTIL2 = new HBaseTestingUtility();
-  protected static final Configuration CONF1 = UTIL1.getConfiguration();
-  protected static final Configuration CONF2 = UTIL2.getConfiguration();
+  protected static Configuration CONF1 = UTIL1.getConfiguration();
+  protected static Configuration CONF2 = UTIL2.getConfiguration();
 
   protected static final int NUM_SLAVES1 = 2;
   protected static final int NUM_SLAVES2 = 4;
@@ -209,13 +210,27 @@ public class TestReplicationBase {
     conf2.setBoolean("hbase.tests.use.shortcircuit.reads", false);
   }
 
-  protected static void restartHBaseCluster(HBaseTestingUtility util, int numSlaves)
+  static void restartSourceCluster(int numSlaves)
       throws Exception {
-    util.shutdownMiniHBaseCluster();
-    util.restartHBaseCluster(numSlaves);
+    IOUtils.closeQuietly(hbaseAdmin, htable1);
+    UTIL1.shutdownMiniHBaseCluster();
+    UTIL1.restartHBaseCluster(numSlaves);
+    // Invalidate the cached connection state.
+    CONF1 = UTIL1.getConfiguration();
+    hbaseAdmin = UTIL1.getAdmin();
+    Connection connection1 = UTIL1.getConnection();
+    htable1 = connection1.getTable(tableName);
+  }
+
+  static void restartTargetHBaseCluster(int numSlaves) throws Exception {
+    IOUtils.closeQuietly(htable2);
+    UTIL2.restartHBaseCluster(numSlaves);
+    // Invalidate the cached connection state
+    CONF2 = UTIL2.getConfiguration();
+    htable2 = UTIL2.getConnection().getTable(tableName);
   }
 
-  protected static void startClusters() throws Exception {
+  private static void startClusters() throws Exception {
     UTIL1.startMiniZKCluster();
     MiniZooKeeperCluster miniZK = UTIL1.getZkCluster();
     LOG.info("Setup first Zk");
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationDisableInactivePeer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationDisableInactivePeer.java
index 4a5cfc1..4ea0bcf 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationDisableInactivePeer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationDisableInactivePeer.java
@@ -21,7 +21,6 @@ import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.fail;
 
 import org.apache.hadoop.hbase.HBaseClassTestRule;
-import org.apache.hadoop.hbase.StartMiniClusterOption;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
@@ -66,8 +65,7 @@ public class TestReplicationDisableInactivePeer extends TestReplicationBase {
 
     // disable and start the peer
     hbaseAdmin.disableReplicationPeer("2");
-    StartMiniClusterOption option = StartMiniClusterOption.builder().numRegionServers(2).build();
-    UTIL2.startMiniHBaseCluster(option);
+    restartTargetHBaseCluster(2);
     Get get = new Get(rowkey);
     for (int i = 0; i < NB_RETRIES; i++) {
       Result res = htable2.get(get);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationDroppedTables.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationDroppedTables.java
index 1d391d3..2d039b0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationDroppedTables.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationDroppedTables.java
@@ -121,20 +121,13 @@ public class TestReplicationDroppedTables extends TestReplicationBase {
   @Test
   public void testEditsDroppedWithDroppedTableNS() throws Exception {
     // also try with a namespace
-    Connection connection1 = ConnectionFactory.createConnection(CONF1);
-    try (Admin admin1 = connection1.getAdmin()) {
-      admin1.createNamespace(NamespaceDescriptor.create("NS").build());
-    }
-    Connection connection2 = ConnectionFactory.createConnection(CONF2);
-    try (Admin admin2 = connection2.getAdmin()) {
-      admin2.createNamespace(NamespaceDescriptor.create("NS").build());
-    }
-    testEditsBehindDroppedTable(true, "NS:test_dropped");
-    try (Admin admin1 = connection1.getAdmin()) {
-      admin1.deleteNamespace("NS");
-    }
-    try (Admin admin2 = connection2.getAdmin()) {
-      admin2.deleteNamespace("NS");
+    UTIL1.getAdmin().createNamespace(NamespaceDescriptor.create("NS").build());
+    UTIL2.getAdmin().createNamespace(NamespaceDescriptor.create("NS").build());
+    try {
+      testEditsBehindDroppedTable(true, "NS:test_dropped");
+    } finally {
+      UTIL1.getAdmin().deleteNamespace("NS");
+      UTIL2.getAdmin().deleteNamespace("NS");
     }
   }
 
@@ -148,8 +141,7 @@ public class TestReplicationDroppedTables extends TestReplicationBase {
 
     // make sure we have a single region server only, so that all
     // edits for all tables go there
-    UTIL1.shutdownMiniHBaseCluster();
-    UTIL1.startMiniHBaseCluster();
+    restartSourceCluster(1);
 
     TableName tablename = TableName.valueOf(tName);
     byte[] familyName = Bytes.toBytes("fam");
@@ -161,8 +153,8 @@ public class TestReplicationDroppedTables extends TestReplicationBase {
                 .newBuilder(familyName).setScope(HConstants.REPLICATION_SCOPE_GLOBAL).build())
             .build();
 
-    Connection connection1 = ConnectionFactory.createConnection(CONF1);
-    Connection connection2 = ConnectionFactory.createConnection(CONF2);
+    Connection connection1 = ConnectionFactory.createConnection(UTIL1.getConfiguration());
+    Connection connection2 = ConnectionFactory.createConnection(UTIL2.getConfiguration());
     try (Admin admin1 = connection1.getAdmin()) {
       admin1.createTable(table);
     }
@@ -223,8 +215,7 @@ public class TestReplicationDroppedTables extends TestReplicationBase {
 
     // make sure we have a single region server only, so that all
     // edits for all tables go there
-    UTIL1.shutdownMiniHBaseCluster();
-    UTIL1.startMiniHBaseCluster();
+    restartSourceCluster(1);
 
     TableName tablename = TableName.valueOf("testdroppedtimed");
     byte[] familyName = Bytes.toBytes("fam");
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatusAfterLagging.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatusAfterLagging.java
index edeaf9d..79520d5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatusAfterLagging.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatusAfterLagging.java
@@ -45,7 +45,7 @@ public class TestReplicationStatusAfterLagging extends TestReplicationBase {
   @Test
   public void testReplicationStatusAfterLagging() throws Exception {
     UTIL2.shutdownMiniHBaseCluster();
-    restartHBaseCluster(UTIL1, 1);
+    restartSourceCluster(1);
     // add some values to cluster 1
     for (int i = 0; i < NB_ROWS_IN_BATCH; i++) {
       Put p = new Put(Bytes.toBytes("row" + i));
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatusBothNormalAndRecoveryLagging.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatusBothNormalAndRecoveryLagging.java
index 16d3822..26f836c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatusBothNormalAndRecoveryLagging.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatusBothNormalAndRecoveryLagging.java
@@ -52,7 +52,7 @@ public class TestReplicationStatusBothNormalAndRecoveryLagging extends TestRepli
       htable1.put(p);
     }
     Thread.sleep(10000);
-    restartHBaseCluster(UTIL1, 1);
+    restartSourceCluster(1);
     Admin hbaseAdmin = UTIL1.getAdmin();
     ServerName serverName = UTIL1.getHBaseCluster().getRegionServer(0).getServerName();
     Thread.sleep(10000);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatusSourceStartedTargetStoppedNewOp.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatusSourceStartedTargetStoppedNewOp.java
index 6deb095..243e245 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatusSourceStartedTargetStoppedNewOp.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatusSourceStartedTargetStoppedNewOp.java
@@ -46,7 +46,7 @@ public class TestReplicationStatusSourceStartedTargetStoppedNewOp extends TestRe
   @Test
   public void testReplicationStatusSourceStartedTargetStoppedNewOp() throws Exception {
     UTIL2.shutdownMiniHBaseCluster();
-    restartHBaseCluster(UTIL1, 1);
+    restartSourceCluster(1);
     Admin hbaseAdmin = UTIL1.getAdmin();
     // add some values to source cluster
     for (int i = 0; i < NB_ROWS_IN_BATCH; i++) {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatusSourceStartedTargetStoppedNoOps.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatusSourceStartedTargetStoppedNoOps.java
index 01f49f4..24c5051 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatusSourceStartedTargetStoppedNoOps.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatusSourceStartedTargetStoppedNoOps.java
@@ -43,7 +43,7 @@ public class TestReplicationStatusSourceStartedTargetStoppedNoOps extends TestRe
   @Test
   public void testReplicationStatusSourceStartedTargetStoppedNoOps() throws Exception {
     UTIL2.shutdownMiniHBaseCluster();
-    restartHBaseCluster(UTIL1, 1);
+    restartSourceCluster(1);
     Admin hbaseAdmin = UTIL1.getAdmin();
     ServerName serverName = UTIL1.getHBaseCluster().getRegionServer(0).getServerName();
     Thread.sleep(10000);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatusSourceStartedTargetStoppedWithRecovery.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatusSourceStartedTargetStoppedWithRecovery.java
index fde87bc..0f3450e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatusSourceStartedTargetStoppedWithRecovery.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatusSourceStartedTargetStoppedWithRecovery.java
@@ -54,7 +54,7 @@ public class TestReplicationStatusSourceStartedTargetStoppedWithRecovery
       htable1.put(p);
     }
     Thread.sleep(10000);
-    restartHBaseCluster(UTIL1, 1);
+    restartSourceCluster(1);
     Admin hbaseAdmin = UTIL1.getAdmin();
     ServerName serverName = UTIL1.getHBaseCluster().getRegionServer(0).getServerName();
     Thread.sleep(10000);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java
index c3bbca9..fa6109a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java
@@ -144,7 +144,7 @@ public class TestReplicationSyncUpTool extends TestReplicationSyncUpToolBase {
 
   private void mimicSyncUpAfterDelete() throws Exception {
     LOG.debug("mimicSyncUpAfterDelete");
-    UTIL2.shutdownMiniHBaseCluster();
+    shutDownTargetHBaseCluster();
 
     List<Delete> list = new ArrayList<>();
     // delete half of the rows
@@ -170,8 +170,8 @@ public class TestReplicationSyncUpTool extends TestReplicationSyncUpToolBase {
     assertEquals("t2_syncup has 101 rows on source, after remove 100 of the replicated colfam", 101,
       rowCount_ht2Source);
 
-    UTIL1.shutdownMiniHBaseCluster();
-    UTIL2.restartHBaseCluster(1);
+    shutDownSourceHBaseCluster();
+    restartTargetHBaseCluster(1);
 
     Thread.sleep(SLEEP_TIME);
 
@@ -189,7 +189,7 @@ public class TestReplicationSyncUpTool extends TestReplicationSyncUpToolBase {
       if (i == NB_RETRIES - 1) {
         if (rowCountHt1TargetAtPeer1 != 50 || rowCountHt2TargetAtPeer1 != 100) {
           // syncUP still failed. Let's look at the source in case anything wrong there
-          UTIL1.restartHBaseCluster(1);
+          restartSourceHBaseCluster(1);
           rowCount_ht1Source = countRows(ht1Source);
           LOG.debug("t1_syncup should have 51 rows at source, and it is " + rowCount_ht1Source);
           rowCount_ht2Source = countRows(ht2Source);
@@ -213,8 +213,8 @@ public class TestReplicationSyncUpTool extends TestReplicationSyncUpToolBase {
 
   private void mimicSyncUpAfterPut() throws Exception {
     LOG.debug("mimicSyncUpAfterPut");
-    UTIL1.restartHBaseCluster(1);
-    UTIL2.shutdownMiniHBaseCluster();
+    restartSourceHBaseCluster(1);
+    shutDownTargetHBaseCluster();
 
     Put p;
     // another 100 + 1 row to t1_syncup
@@ -244,8 +244,8 @@ public class TestReplicationSyncUpTool extends TestReplicationSyncUpToolBase {
     int rowCount_ht2Source = countRows(ht2Source);
     assertEquals("t2_syncup has 202 rows on source", 202, rowCount_ht2Source);
 
-    UTIL1.shutdownMiniHBaseCluster();
-    UTIL2.restartHBaseCluster(1);
+    shutDownSourceHBaseCluster();
+    restartTargetHBaseCluster(1);
 
     Thread.sleep(SLEEP_TIME);
 
@@ -265,7 +265,7 @@ public class TestReplicationSyncUpTool extends TestReplicationSyncUpToolBase {
       if (i == NB_RETRIES - 1) {
         if (rowCountHt1TargetAtPeer1 != 100 || rowCountHt2TargetAtPeer1 != 200) {
           // syncUP still failed. Let's look at the source in case anything wrong there
-          UTIL1.restartHBaseCluster(1);
+          restartSourceHBaseCluster(1);
           rowCount_ht1Source = countRows(ht1Source);
           LOG.debug("t1_syncup should have 102 rows at source, and it is " + rowCount_ht1Source);
           rowCount_ht2Source = countRows(ht2Source);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpToolBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpToolBase.java
index bf3941d..ee5276d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpToolBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpToolBase.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.replication;
 
 import static org.apache.hadoop.hbase.HConstants.REPLICATION_SCOPE_GLOBAL;
 
+import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.TableName;
@@ -108,7 +109,7 @@ public abstract class TestReplicationSyncUpToolBase {
     UTIL1.shutdownMiniCluster();
   }
 
-  protected final void setupReplication() throws Exception {
+  final void setupReplication() throws Exception {
     Admin admin1 = UTIL1.getAdmin();
     admin1.createTable(t1SyncupSource);
     admin1.createTable(t2SyncupSource);
@@ -135,7 +136,33 @@ public abstract class TestReplicationSyncUpToolBase {
     admin1.addReplicationPeer("1", rpc);
   }
 
-  protected final void syncUp(HBaseTestingUtility util) throws Exception {
+  final void syncUp(HBaseTestingUtility util) throws Exception {
     ToolRunner.run(util.getConfiguration(), new ReplicationSyncUp(), new String[0]);
   }
+
+  // Utilities that manager shutdown / restart of source / sink clusters. They take care of
+  // invalidating stale connections after shutdown / restarts.
+  final void shutDownSourceHBaseCluster() throws Exception {
+    IOUtils.closeQuietly(ht1Source, ht2Source);
+    UTIL1.shutdownMiniHBaseCluster();
+  }
+
+  final void shutDownTargetHBaseCluster() throws Exception {
+    IOUtils.closeQuietly(ht1TargetAtPeer1, ht2TargetAtPeer1);
+    UTIL2.shutdownMiniHBaseCluster();
+  }
+
+  final void restartSourceHBaseCluster(int numServers) throws Exception {
+    IOUtils.closeQuietly(ht1Source, ht2Source);
+    UTIL1.restartHBaseCluster(numServers);
+    ht1Source = UTIL1.getConnection().getTable(TN1);
+    ht2Source = UTIL1.getConnection().getTable(TN2);
+  }
+
+  final void restartTargetHBaseCluster(int numServers) throws Exception {
+    IOUtils.closeQuietly(ht1TargetAtPeer1, ht2TargetAtPeer1);
+    UTIL2.restartHBaseCluster(numServers);
+    ht1TargetAtPeer1 = UTIL2.getConnection().getTable(TN1);
+    ht2TargetAtPeer1 = UTIL2.getConnection().getTable(TN2);
+  }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpToolWithBulkLoadedData.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpToolWithBulkLoadedData.java
index 6247c22..5c4fc91 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpToolWithBulkLoadedData.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpToolWithBulkLoadedData.java
@@ -112,7 +112,7 @@ public class TestReplicationSyncUpToolWithBulkLoadedData extends TestReplication
   private void mimicSyncUpAfterBulkLoad(Iterator<String> randomHFileRangeListIterator)
       throws Exception {
     LOG.debug("mimicSyncUpAfterBulkLoad");
-    UTIL2.shutdownMiniHBaseCluster();
+    shutDownTargetHBaseCluster();
 
     loadAndReplicateHFiles(false, randomHFileRangeListIterator);
 
@@ -124,8 +124,8 @@ public class TestReplicationSyncUpToolWithBulkLoadedData extends TestReplication
     assertEquals("t2_syncup has 406 rows on source, after bulk load of another 203 hfiles", 406,
       rowCount_ht2Source);
 
-    UTIL1.shutdownMiniHBaseCluster();
-    UTIL2.restartHBaseCluster(1);
+    shutDownSourceHBaseCluster();
+    restartTargetHBaseCluster(1);
 
     Thread.sleep(SLEEP_TIME);
 
@@ -146,7 +146,7 @@ public class TestReplicationSyncUpToolWithBulkLoadedData extends TestReplication
       if (i == NB_RETRIES - 1) {
         if (rowCountHt1TargetAtPeer1 != 200 || rowCountHt2TargetAtPeer1 != 400) {
           // syncUP still failed. Let's look at the source in case anything wrong there
-          UTIL1.restartHBaseCluster(1);
+          restartSourceHBaseCluster(1);
           rowCount_ht1Source = countRows(ht1Source);
           LOG.debug("t1_syncup should have 206 rows at source, and it is " + rowCount_ht1Source);
           rowCount_ht2Source = countRows(ht2Source);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java
index 953b0c9..af41e05 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.RegionLocations;
+import org.apache.hadoop.hbase.StartMiniClusterOption;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.ClusterConnectionFactory;
@@ -114,7 +115,9 @@ public class TestRegionReplicaReplicationEndpointNoMaster {
     }
     HTU.getConfiguration().set(CoprocessorHost.WAL_COPROCESSOR_CONF_KEY,
       walCoprocs);
-    HTU.startMiniCluster(NB_SERVERS);
+    StartMiniClusterOption option = StartMiniClusterOption.builder().numAlwaysStandByMasters(1).
+        numRegionServers(NB_SERVERS).numDataNodes(NB_SERVERS).build();
+    HTU.startMiniCluster(option);
 
     // Create table then get the single region for our new table.
     HTableDescriptor htd = HTU.createTableDescriptor(tableName.getNameAsString());
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/SecureTestCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/SecureTestCluster.java
index 2263bde..964ef14 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/SecureTestCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/SecureTestCluster.java
@@ -80,8 +80,8 @@ public class SecureTestCluster {
   public static void tearDown() throws Exception {
     if (CLUSTER != null) {
       CLUSTER.shutdown();
+      CLUSTER.join();
     }
-    CLUSTER.join();
     if (KDC != null) {
       KDC.stop();
     }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestDelegationTokenWithEncryption.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestDelegationTokenWithEncryption.java
index 9353576..0a13df0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestDelegationTokenWithEncryption.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestDelegationTokenWithEncryption.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.security.token;
 
 import static org.junit.Assert.assertArrayEquals;
-
 import java.util.Arrays;
 import java.util.Collection;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
@@ -32,7 +31,6 @@ import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.ipc.BlockingRpcClient;
 import org.apache.hadoop.hbase.ipc.NettyRpcClient;
 import org.apache.hadoop.hbase.ipc.RpcClientFactory;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -73,8 +71,10 @@ public class TestDelegationTokenWithEncryption extends SecureTestCluster {
   }
 
   @Parameters(name = "{index}: rpcClientImpl={0}")
-  public static Collection<Object[]> parameters() {
-    return Arrays.asList(new Object[] { BlockingRpcClient.class.getName() },
+  public static Collection<Object> parameters() {
+    // Client connection supports only non-blocking RPCs (due to master registry restriction), hence
+    // we only test NettyRpcClient.
+    return Arrays.asList(
       new Object[] { NettyRpcClient.class.getName() });
   }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestGenerateDelegationToken.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestGenerateDelegationToken.java
index ee5f18f..bb6208d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestGenerateDelegationToken.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestGenerateDelegationToken.java
@@ -21,7 +21,6 @@ import static org.hamcrest.CoreMatchers.containsString;
 import static org.hamcrest.CoreMatchers.instanceOf;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
-
 import com.google.protobuf.ServiceException;
 import java.io.IOException;
 import java.util.Arrays;
@@ -32,7 +31,6 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.ipc.BlockingRpcClient;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
 import org.apache.hadoop.hbase.ipc.NettyRpcClient;
 import org.apache.hadoop.hbase.ipc.RpcClientFactory;
@@ -76,8 +74,10 @@ public class TestGenerateDelegationToken extends SecureTestCluster {
   }
 
   @Parameters(name = "{index}: rpcClientImpl={0}")
-  public static Collection<Object[]> parameters() {
-    return Arrays.asList(new Object[] { BlockingRpcClient.class.getName() },
+  public static Collection<Object> parameters() {
+    // Client connection supports only non-blocking RPCs (due to master registry restriction), hence
+    // we only test NettyRpcClient.
+    return Arrays.asList(
       new Object[] { NettyRpcClient.class.getName() });
   }
 
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 a2981fb..d0fbd3f 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
@@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.ClusterId;
 import org.apache.hadoop.hbase.CoordinatedStateManager;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 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;
@@ -382,6 +383,10 @@ public class TestTokenAuthentication {
   @Before
   public void setUp() throws Exception {
     TEST_UTIL = new HBaseTestingUtility();
+    // Override the connection registry to avoid spinning up a mini cluster for the connection below
+    // to go through.
+    TEST_UTIL.getConfiguration().set(HConstants.CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY,
+        HConstants.ZK_CONNECTION_REGISTRY_CLASS);
     TEST_UTIL.startMiniZKCluster();
     // register token type for protocol
     SecurityInfo.addInfo(AuthenticationProtos.AuthenticationService.getDescriptor().getName(),