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/02/03 18:35:48 UTC

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

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 030996849d8e79ad156a59d450a69143e2235a85
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 ...