You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2019/05/14 18:56:20 UTC

[hbase] branch master updated: HBASE-22377 Provide API to check the existence of a namespace which does not require ADMIN permissions (#225)

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

apurtell pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/master by this push:
     new a9de9f5  HBASE-22377 Provide API to check the existence of a namespace which does not require ADMIN permissions (#225)
a9de9f5 is described below

commit a9de9f5f28124072ededc5401e4d45dd32860208
Author: Andrew Purtell <ap...@apache.org>
AuthorDate: Tue May 14 11:56:16 2019 -0700

    HBASE-22377 Provide API to check the existence of a namespace which does not require ADMIN permissions (#225)
    
    Signed-off-by: Xu Cang <xu...@apache.org>
---
 .../java/org/apache/hadoop/hbase/client/Admin.java |   25 +-
 .../org/apache/hadoop/hbase/client/AsyncAdmin.java |    6 +
 .../hadoop/hbase/client/AsyncHBaseAdmin.java       |    5 +
 .../hbase/client/ConnectionImplementation.java     |    7 +
 .../org/apache/hadoop/hbase/client/HBaseAdmin.java |   34 +-
 .../hadoop/hbase/client/RawAsyncHBaseAdmin.java    |   14 +
 .../hbase/client/ShortCircuitMasterConnection.java |    9 +
 .../src/main/protobuf/Master.proto                 |   13 +-
 .../hadoop/hbase/coprocessor/MasterObserver.java   |   18 +
 .../org/apache/hadoop/hbase/master/HMaster.java    |   19 +
 .../hadoop/hbase/master/MasterCoprocessorHost.java |   18 +
 .../hadoop/hbase/master/MasterRpcServices.java     |   15 +
 .../hbase/security/access/AccessController.java    |    6 +
 .../org/apache/hadoop/hbase/TestNamespace.java     |    3 +
 .../hbase/client/TestAsyncNamespaceAdminApi.java   |    2 +
 .../hbase/coprocessor/TestMasterObserver.java      |   22 +
 hbase-shell/src/main/ruby/hbase/admin.rb           |    2 +-
 .../hbase/thrift2/ThriftHBaseServiceHandler.java   |   14 +
 .../hadoop/hbase/thrift2/client/ThriftAdmin.java   |   10 +
 .../hbase/thrift2/generated/THBaseService.java     | 4712 ++++++++++++--------
 .../org/apache/hadoop/hbase/thrift2/hbase.thrift   |    6 +
 21 files changed, 3059 insertions(+), 1901 deletions(-)

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
index 675942a..c7d60c5 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
@@ -112,7 +112,7 @@ public interface Admin extends Abortable, Closeable {
   /**
    * @param tableName Table to check.
    * @return <code>true</code> if table exists already.
-   * @throws IOException
+   * @throws IOException if a remote or network exception occurs
    */
   boolean tableExists(TableName tableName) throws IOException;
 
@@ -486,7 +486,7 @@ public interface Admin extends Abortable, Closeable {
    * Get all the online regions on a region server.
    *
    * @return List of {@link RegionInfo}
-   * @throws java.io.IOException
+   * @throws IOException if a remote or network exception occurs
    */
   List<RegionInfo> getRegions(ServerName serverName) throws IOException;
 
@@ -1130,8 +1130,18 @@ public interface Admin extends Abortable, Closeable {
       throws NamespaceNotFoundException, IOException;
 
   /**
-   * List available namespace descriptors.
+   * List available namespaces
+   *
+   * @return List of namespace names
+   * @throws IOException if a remote or network exception occurs
+   */
+  String[] listNamespaces() throws IOException;
+
+  /**
+   * List available namespace descriptors
+   *
    * @return List of descriptors
+   * @throws IOException if a remote or network exception occurs
    */
   NamespaceDescriptor[] listNamespaceDescriptors() throws IOException;
 
@@ -1139,6 +1149,7 @@ public interface Admin extends Abortable, Closeable {
    * Get list of table descriptors by namespace.
    * @param name namespace name
    * @return returns a list of TableDescriptors
+   * @throws IOException if a remote or network exception occurs
    */
   List<TableDescriptor> listTableDescriptorsByNamespace(byte[] name) throws IOException;
 
@@ -1146,6 +1157,7 @@ public interface Admin extends Abortable, Closeable {
    * Get list of table names by namespace.
    * @param name namespace name
    * @return The list of table names in the namespace
+   * @throws IOException if a remote or network exception occurs
    */
   TableName[] listTableNamesByNamespace(String name) throws IOException;
 
@@ -1154,6 +1166,7 @@ public interface Admin extends Abortable, Closeable {
    *
    * @param tableName the name of the table
    * @return List of {@link RegionInfo}.
+   * @throws IOException if a remote or network exception occurs
    */
   List<RegionInfo> getRegions(TableName tableName) throws IOException;
 
@@ -1178,7 +1191,7 @@ public interface Admin extends Abortable, Closeable {
    * @param mayInterruptIfRunning if the proc completed at least one step, should it be aborted?
    * @return <code>true</code> if aborted, <code>false</code> if procedure already completed or does
    *         not exist
-   * @throws IOException
+   * @throws IOException if a remote or network exception occurs
    * @deprecated Since 2.1.1 -- to be removed.
    */
   @Deprecated
@@ -1198,7 +1211,7 @@ public interface Admin extends Abortable, Closeable {
    * @param procId ID of the procedure to abort
    * @param mayInterruptIfRunning if the proc completed at least one step, should it be aborted?
    * @return <code>true</code> if aborted, <code>false</code> if procedure already completed or does not exist
-   * @throws IOException
+   * @throws IOException if a remote or network exception occurs
    * @deprecated Since 2.1.1 -- to be removed.
    */
   @Deprecated
@@ -1208,7 +1221,7 @@ public interface Admin extends Abortable, Closeable {
   /**
    * Get procedures.
    * @return procedure list in JSON
-   * @throws IOException
+   * @throws IOException if a remote or network exception occurs
    */
   String getProcedures() throws IOException;
 
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
index 96ea677..0f66977 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
@@ -271,6 +271,12 @@ public interface AsyncAdmin {
   CompletableFuture<NamespaceDescriptor> getNamespaceDescriptor(String name);
 
   /**
+   * List available namespaces
+   * @return List of namespaces wrapped by a {@link CompletableFuture}.
+   */
+  CompletableFuture<List<String>> listNamespaces();
+
+  /**
    * List available namespace descriptors
    * @return List of descriptors wrapped by a {@link CompletableFuture}.
    */
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
index 1566110..7787f18 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
@@ -215,6 +215,11 @@ class AsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
+  public CompletableFuture<List<String>> listNamespaces() {
+    return wrap(rawAdmin.listNamespaces());
+  }
+
+  @Override
   public CompletableFuture<List<NamespaceDescriptor>> listNamespaceDescriptors() {
     return wrap(rawAdmin.listNamespaceDescriptors());
   }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
index 2954e04..e63c16d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
@@ -1562,6 +1562,13 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
       }
 
       @Override
+      public MasterProtos.ListNamespacesResponse listNamespaces(
+          RpcController controller,
+          MasterProtos.ListNamespacesRequest request) throws ServiceException {
+        return stub.listNamespaces(controller, request);
+      }
+
+      @Override
       public MasterProtos.GetNamespaceDescriptorResponse getNamespaceDescriptor(
           RpcController controller,
           MasterProtos.GetNamespaceDescriptorRequest request) throws ServiceException {
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
index 17dfa6e..5812bd8 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
@@ -182,6 +182,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshot
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDecommissionedRegionServersRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespacesRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest;
@@ -1412,7 +1413,7 @@ public class HBaseAdmin implements Admin {
    * @param nameOfRegionB encoded or full name of region b
    * @param forcible true if do a compulsory merge, otherwise we will only merge
    *          two adjacent regions
-   * @throws IOException
+   * @throws IOException if a remote or network exception occurs
    */
   @VisibleForTesting
   public void mergeRegionsSync(
@@ -1512,7 +1513,7 @@ public class HBaseAdmin implements Admin {
    *   Therefore, this is for internal testing only.
    * @param regionName encoded or full name of region
    * @param splitPoint key where region splits
-   * @throws IOException
+   * @throws IOException if a remote or network exception occurs
    */
   @VisibleForTesting
   public void splitRegionSync(byte[] regionName, byte[] splitPoint) throws IOException {
@@ -1526,7 +1527,7 @@ public class HBaseAdmin implements Admin {
    * @param splitPoint split point
    * @param timeout how long to wait on split
    * @param units time units
-   * @throws IOException
+   * @throws IOException if a remote or network exception occurs
    */
   public void splitRegionSync(byte[] regionName, byte[] splitPoint, final long timeout,
       final TimeUnit units) throws IOException {
@@ -1647,7 +1648,7 @@ public class HBaseAdmin implements Admin {
    *  MetaTableAccessor#getRegionLocation(Connection, byte[])}
    *  else null.
    * Throw IllegalArgumentException if <code>regionName</code> is null.
-   * @throws IOException
+   * @throws IOException if a remote or network exception occurs
    */
   Pair<RegionInfo, ServerName> getRegion(final byte[] regionName) throws IOException {
     if (regionName == null) {
@@ -1921,6 +1922,29 @@ public class HBaseAdmin implements Admin {
     });
   }
 
+  /**
+   * List available namespaces
+   * @return List of namespace names
+   * @throws IOException if a remote or network exception occurs
+   */
+  @Override
+  public String[] listNamespaces() throws IOException {
+    return executeCallable(new MasterCallable<String[]>(getConnection(),
+        getRpcControllerFactory()) {
+      @Override
+      protected String[] rpcCall() throws Exception {
+        List<String> list = master.listNamespaces(getRpcController(),
+          ListNamespacesRequest.newBuilder().build()).getNamespaceNameList();
+        return list.toArray(new String[list.size()]);
+      }
+    });
+  }
+
+  /**
+   * List available namespace descriptors
+   * @return List of descriptors
+   * @throws IOException if a remote or network exception occurs
+   */
   @Override
   public NamespaceDescriptor[] listNamespaceDescriptors() throws IOException {
     return executeCallable(new MasterCallable<NamespaceDescriptor[]>(getConnection(),
@@ -3594,7 +3618,7 @@ public class HBaseAdmin implements Admin {
    * </ol>
    * @param tableName name of the table to sync to the peer
    * @param splits table split keys
-   * @throws IOException
+   * @throws IOException if a remote or network exception occurs
    */
   private void checkAndSyncTableDescToPeers(final TableName tableName, final byte[][] splits)
       throws IOException {
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
index 0fd3cba..3303fd3 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
@@ -214,6 +214,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDecomm
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDecommissionedRegionServersResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespacesRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespacesResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest;
@@ -829,6 +831,18 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
+  public CompletableFuture<List<String>> listNamespaces() {
+    return this
+        .<List<String>> newMasterCaller()
+        .action(
+          (controller, stub) -> this
+              .<ListNamespacesRequest, ListNamespacesResponse, List<String>> call(
+                controller, stub, ListNamespacesRequest.newBuilder().build(), (s, c, req,
+                  done) -> s.listNamespaces(c, req, done),
+                (resp) -> resp.getNamespaceNameList())).call();
+  }
+
+  @Override
   public CompletableFuture<List<NamespaceDescriptor>> listNamespaceDescriptors() {
     return this
         .<List<NamespaceDescriptor>> newMasterCaller()
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java
index 8927615..8d6da40 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.Rev
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.RevokeResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse;
+
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnRequest;
@@ -107,6 +108,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDecomm
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDecommissionedRegionServersResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespacesRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespacesResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest;
@@ -447,6 +450,12 @@ public class ShortCircuitMasterConnection implements MasterKeepAliveConnection {
   }
 
   @Override
+  public ListNamespacesResponse listNamespaces(RpcController controller,
+      ListNamespacesRequest request) throws ServiceException {
+    return stub.listNamespaces(controller, request);
+  }
+
+  @Override
   public MajorCompactionTimestampResponse getLastMajorCompactionTimestampForRegion(
       RpcController controller, MajorCompactionTimestampForRegionRequest request)
       throws ServiceException {
diff --git a/hbase-protocol-shaded/src/main/protobuf/Master.proto b/hbase-protocol-shaded/src/main/protobuf/Master.proto
index 369fcfb..342a782 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Master.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Master.proto
@@ -237,6 +237,13 @@ message GetNamespaceDescriptorResponse {
   required NamespaceDescriptor namespaceDescriptor = 1;
 }
 
+message ListNamespacesRequest {
+}
+
+message ListNamespacesResponse {
+  repeated string namespaceName = 1;
+}
+
 message ListNamespaceDescriptorsRequest {
 }
 
@@ -916,7 +923,7 @@ service MasterService {
   rpc GetNamespaceDescriptor(GetNamespaceDescriptorRequest)
     returns(GetNamespaceDescriptorResponse);
 
-  /** returns a list of namespaces */
+  /** returns a list of namespace descriptors */
   rpc ListNamespaceDescriptors(ListNamespaceDescriptorsRequest)
     returns(ListNamespaceDescriptorsResponse);
 
@@ -1035,6 +1042,10 @@ service MasterService {
   rpc GetUserPermissions (GetUserPermissionsRequest) returns (GetUserPermissionsResponse);
 
   rpc HasUserPermissions (HasUserPermissionsRequest) returns (HasUserPermissionsResponse);
+
+  /** returns a list of namespace names */
+  rpc ListNamespaces(ListNamespacesRequest)
+    returns(ListNamespacesResponse);
 }
 
 // HBCK Service definitions.
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
index 88b01a2..b29ed0b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
@@ -964,6 +964,24 @@ public interface MasterObserver {
       NamespaceDescriptor ns) throws IOException {}
 
   /**
+   * Called before a listNamespaces request has been processed.
+   * @param ctx the environment to interact with the framework and master
+   * @param namespaces an empty list, can be filled with what to return if bypassing
+   * @throws IOException if something went wrong
+   */
+  default void preListNamespaces(ObserverContext<MasterCoprocessorEnvironment> ctx,
+      List<String> namespaces) throws IOException {}
+
+  /**
+   * Called after a listNamespaces request has been processed.
+   * @param ctx the environment to interact with the framework and master
+   * @param namespaces the list of namespaces about to be returned
+   * @throws IOException if something went wrong
+   */
+  default void postListNamespaces(ObserverContext<MasterCoprocessorEnvironment> ctx,
+      List<String> namespaces)  throws IOException {};
+
+  /**
    * Called before a listNamespaceDescriptors request has been processed.
    * @param ctx the environment to interact with the framework and master
    * @param descriptors an empty list, can be filled with what to return by coprocessor
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index 8eb8a48..11a700a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -3392,6 +3392,25 @@ public class HMaster extends HRegionServer implements MasterServices {
     return nsds;
   }
 
+  /**
+   * List namespace names
+   * @return All namespace names
+   */
+  public List<String> listNamespaces() throws IOException {
+    checkInitialized();
+    List<String> namespaces = new ArrayList<>();
+    if (cpHost != null) {
+      cpHost.preListNamespaces(namespaces);
+    }
+    for (NamespaceDescriptor namespace : clusterSchemaService.getNamespaces()) {
+      namespaces.add(namespace.getName());
+    }
+    if (cpHost != null) {
+      cpHost.postListNamespaces(namespaces);
+    }
+    return namespaces;
+  }
+
   @Override
   public List<TableName> listTableNamesByNamespace(String name) throws IOException {
     checkInitialized();
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
index 720f496..1ee548b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
@@ -297,6 +297,24 @@ public class MasterCoprocessorHost
     });
   }
 
+  public void preListNamespaces(final List<String> namespaces) throws IOException {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
+      @Override
+      public void call(MasterObserver oserver) throws IOException {
+        oserver.preListNamespaces(this, namespaces);
+      }
+    });
+  }
+
+  public void postListNamespaces(final List<String> namespaces) throws IOException {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
+      @Override
+      public void call(MasterObserver oserver) throws IOException {
+        oserver.postListNamespaces(this, namespaces);
+      }
+    });
+  }
+
   public void preListNamespaceDescriptors(final List<NamespaceDescriptor> descriptors)
       throws IOException {
     execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index cd017d8..b772fd7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -233,6 +233,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDecomm
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDecommissionedRegionServersResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespacesRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespacesResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest;
@@ -995,6 +997,19 @@ public class MasterRpcServices extends RSRpcServices
   }
 
   @Override
+  public ListNamespacesResponse listNamespaces(
+      RpcController controller, ListNamespacesRequest request)
+      throws ServiceException {
+    try {
+      return ListNamespacesResponse.newBuilder()
+        .addAllNamespaceName(master.listNamespaces())
+        .build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
   public GetNamespaceDescriptorResponse getNamespaceDescriptor(
       RpcController controller, GetNamespaceDescriptorRequest request)
       throws ServiceException {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
index 62bcb34..1dfd7e6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
@@ -1176,6 +1176,12 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor,
   }
 
   @Override
+  public void postListNamespaces(ObserverContext<MasterCoprocessorEnvironment> ctx,
+      List<String> namespaces) throws IOException {
+    /* always allow namespace listing */
+  }
+
+  @Override
   public void postListNamespaceDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx,
       List<NamespaceDescriptor> descriptors) throws IOException {
     // Retains only those which passes authorization checks, as the checks weren't done as part
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java
index f062e58..1fe43f1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java
@@ -110,6 +110,7 @@ public class TestNamespace {
     assertNotNull(ns);
     assertEquals(ns.getName(), NamespaceDescriptor.SYSTEM_NAMESPACE.getName());
 
+    assertEquals(2, admin.listNamespaces().length);
     assertEquals(2, admin.listNamespaceDescriptors().length);
 
     //verify existence of system tables
@@ -175,9 +176,11 @@ public class TestNamespace {
 
     //create namespace and verify
     admin.createNamespace(NamespaceDescriptor.create(nsName).build());
+    assertEquals(3, admin.listNamespaces().length);
     assertEquals(3, admin.listNamespaceDescriptors().length);
     //remove namespace and verify
     admin.deleteNamespace(nsName);
+    assertEquals(2, admin.listNamespaces().length);
     assertEquals(2, admin.listNamespaceDescriptors().length);
   }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNamespaceAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNamespaceAdminApi.java
index 8de32b8..e413e8e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNamespaceAdminApi.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNamespaceAdminApi.java
@@ -71,9 +71,11 @@ public class TestAsyncNamespaceAdminApi extends TestAsyncAdminBase {
 
     // create namespace and verify
     admin.createNamespace(NamespaceDescriptor.create(nsName).build()).join();
+    assertEquals(3, admin.listNamespaces().get().size());
     assertEquals(3, admin.listNamespaceDescriptors().get().size());
     // delete namespace and verify
     admin.deleteNamespace(nsName).join();
+    assertEquals(2, admin.listNamespaces().get().size());
     assertEquals(2, admin.listNamespaceDescriptors().get().size());
   }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
index 178429d..1bc3996 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
@@ -111,6 +111,8 @@ public class TestMasterObserver {
     private boolean postModifyNamespaceCalled;
     private boolean preGetNamespaceDescriptorCalled;
     private boolean postGetNamespaceDescriptorCalled;
+    private boolean preListNamespacesCalled;
+    private boolean postListNamespacesCalled;
     private boolean preListNamespaceDescriptorsCalled;
     private boolean postListNamespaceDescriptorsCalled;
     private boolean preAddColumnCalled;
@@ -204,6 +206,8 @@ public class TestMasterObserver {
       postModifyNamespaceCalled = false;
       preGetNamespaceDescriptorCalled = false;
       postGetNamespaceDescriptorCalled = false;
+      preListNamespacesCalled = false;
+      postListNamespacesCalled = false;
       preListNamespaceDescriptorsCalled = false;
       postListNamespaceDescriptorsCalled = false;
       preAddColumnCalled = false;
@@ -473,6 +477,18 @@ public class TestMasterObserver {
     }
 
     @Override
+    public void preListNamespaces(ObserverContext<MasterCoprocessorEnvironment> ctx,
+        List<String> namespaces) {
+      preListNamespacesCalled = true;
+    }
+
+    @Override
+    public void postListNamespaces(ObserverContext<MasterCoprocessorEnvironment> ctx,
+        List<String> namespaces) {
+      postListNamespacesCalled = true;
+    }
+
+    @Override
     public void preListNamespaceDescriptors(ObserverContext<MasterCoprocessorEnvironment> env,
         List<NamespaceDescriptor> descriptors) throws IOException {
       preListNamespaceDescriptorsCalled = true;
@@ -1253,6 +1269,7 @@ public class TestMasterObserver {
         final ObserverContext<MasterCoprocessorEnvironment> ctx,
         final RegionInfo[] regionsToMerge) throws IOException {
     }
+
   }
 
   private static HBaseTestingUtility UTIL = new HBaseTestingUtility();
@@ -1495,6 +1512,11 @@ public class TestMasterObserver {
 
     // create a table
     Admin admin = UTIL.getAdmin();
+
+    admin.listNamespaces();
+    assertTrue("preListNamespaces should have been called", cp.preListNamespacesCalled);
+    assertTrue("postListNamespaces should have been called", cp.postListNamespacesCalled);
+
     admin.createNamespace(NamespaceDescriptor.create(testNamespace).build());
     assertTrue("Test namespace should be created", cp.wasCreateNamespaceCalled());
 
diff --git a/hbase-shell/src/main/ruby/hbase/admin.rb b/hbase-shell/src/main/ruby/hbase/admin.rb
index acfe178..a9f69b5 100644
--- a/hbase-shell/src/main/ruby/hbase/admin.rb
+++ b/hbase-shell/src/main/ruby/hbase/admin.rb
@@ -1244,7 +1244,7 @@ module Hbase
     # Returns a list of namespaces in hbase
     def list_namespace(regex = '.*')
       pattern = java.util.regex.Pattern.compile(regex)
-      list = @admin.listNamespaceDescriptors.map(&:getName)
+      list = @admin.listNamespaces
       list.select { |s| pattern.match(s) }
     end
 
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 309b6ac..8b1be58 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
@@ -794,6 +794,20 @@ public class ThriftHBaseServiceHandler extends HBaseServiceHandler implements TH
   }
 
   @Override
+  public List<String> listNamespaces() throws TIOError, TException {
+    try {
+      String[] namespaces = connectionCache.getAdmin().listNamespaces();
+      List<String> result = new ArrayList<>(namespaces.length);
+      for (String ns: namespaces) {
+        result.add(ns);
+      }
+      return result;
+    } catch (IOException e) {
+      throw getTIOError(e);
+    }
+  }
+
+  @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/ThriftAdmin.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java
index fbb6722..1884fb0 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java
@@ -396,6 +396,16 @@ public class ThriftAdmin implements Admin {
   }
 
   @Override
+  public String[] listNamespaces() throws IOException {
+    try {
+      List<String> tNamespaces = client.listNamespaces();
+      return tNamespaces.toArray(new String[tNamespaces.size()]);
+    } catch (TException e) {
+      throw new IOException(e);
+    }
+  }
+
+  @Override
   public NamespaceDescriptor[] listNamespaceDescriptors() throws IOException {
     try {
       List<TNamespaceDescriptor> tNamespaceDescriptors = client.listNamespaceDescriptors();
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 33888c7..11732fd 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
@@ -494,6 +494,12 @@ public class THBaseService {
      */
     public java.util.List<TNamespaceDescriptor> listNamespaceDescriptors() throws TIOError, org.apache.thrift.TException;
 
+    /**
+     * @return all namespace names
+     * 
+     */
+    public java.util.List<java.lang.String> listNamespaces() throws TIOError, org.apache.thrift.TException;
+
   }
 
   public interface AsyncIface {
@@ -588,6 +594,8 @@ public class THBaseService {
 
     public void listNamespaceDescriptors(org.apache.thrift.async.AsyncMethodCallback<java.util.List<TNamespaceDescriptor>> resultHandler) throws org.apache.thrift.TException;
 
+    public void listNamespaces(org.apache.thrift.async.AsyncMethodCallback<java.util.List<java.lang.String>> resultHandler) throws org.apache.thrift.TException;
+
   }
 
   public static class Client extends org.apache.thrift.TServiceClient implements Iface {
@@ -1775,6 +1783,31 @@ public class THBaseService {
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "listNamespaceDescriptors failed: unknown result");
     }
 
+    public java.util.List<java.lang.String> listNamespaces() throws TIOError, org.apache.thrift.TException
+    {
+      send_listNamespaces();
+      return recv_listNamespaces();
+    }
+
+    public void send_listNamespaces() throws org.apache.thrift.TException
+    {
+      listNamespaces_args args = new listNamespaces_args();
+      sendBase("listNamespaces", args);
+    }
+
+    public java.util.List<java.lang.String> recv_listNamespaces() throws TIOError, org.apache.thrift.TException
+    {
+      listNamespaces_result result = new listNamespaces_result();
+      receiveBase(result, "listNamespaces");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.io != null) {
+        throw result.io;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "listNamespaces 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> {
@@ -3353,6 +3386,35 @@ public class THBaseService {
       }
     }
 
+    public void listNamespaces(org.apache.thrift.async.AsyncMethodCallback<java.util.List<java.lang.String>> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      listNamespaces_call method_call = new listNamespaces_call(resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class listNamespaces_call extends org.apache.thrift.async.TAsyncMethodCall<java.util.List<java.lang.String>> {
+      public listNamespaces_call(org.apache.thrift.async.AsyncMethodCallback<java.util.List<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("listNamespaces", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        listNamespaces_args args = new listNamespaces_args();
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public java.util.List<java.lang.String> getResult() throws TIOError, 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_listNamespaces();
+      }
+    }
+
   }
 
   public static class Processor<I extends Iface> extends org.apache.thrift.TBaseProcessor<I> implements org.apache.thrift.TProcessor {
@@ -3411,6 +3473,7 @@ public class THBaseService {
       processMap.put("deleteNamespace", new deleteNamespace());
       processMap.put("getNamespaceDescriptor", new getNamespaceDescriptor());
       processMap.put("listNamespaceDescriptors", new listNamespaceDescriptors());
+      processMap.put("listNamespaces", new listNamespaces());
       return processMap;
     }
 
@@ -4733,6 +4796,35 @@ public class THBaseService {
       }
     }
 
+    public static class listNamespaces<I extends Iface> extends org.apache.thrift.ProcessFunction<I, listNamespaces_args> {
+      public listNamespaces() {
+        super("listNamespaces");
+      }
+
+      public listNamespaces_args getEmptyArgsInstance() {
+        return new listNamespaces_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      @Override
+      protected boolean rethrowUnhandledExceptions() {
+        return false;
+      }
+
+      public listNamespaces_result getResult(I iface, listNamespaces_args args) throws org.apache.thrift.TException {
+        listNamespaces_result result = new listNamespaces_result();
+        try {
+          result.success = iface.listNamespaces();
+        } catch (TIOError io) {
+          result.io = io;
+        }
+        return result;
+      }
+    }
+
   }
 
   public static class AsyncProcessor<I extends AsyncIface> extends org.apache.thrift.TBaseAsyncProcessor<I> {
@@ -4791,6 +4883,7 @@ public class THBaseService {
       processMap.put("deleteNamespace", new deleteNamespace());
       processMap.put("getNamespaceDescriptor", new getNamespaceDescriptor());
       processMap.put("listNamespaceDescriptors", new listNamespaceDescriptors());
+      processMap.put("listNamespaces", new listNamespaces());
       return processMap;
     }
 
@@ -7720,6 +7813,71 @@ public class THBaseService {
       }
     }
 
+    public static class listNamespaces<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, listNamespaces_args, java.util.List<java.lang.String>> {
+      public listNamespaces() {
+        super("listNamespaces");
+      }
+
+      public listNamespaces_args getEmptyArgsInstance() {
+        return new listNamespaces_args();
+      }
+
+      public org.apache.thrift.async.AsyncMethodCallback<java.util.List<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.util.List<java.lang.String>>() { 
+          public void onComplete(java.util.List<java.lang.String> o) {
+            listNamespaces_result result = new listNamespaces_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;
+            listNamespaces_result result = new listNamespaces_result();
+            if (e instanceof TIOError) {
+              result.io = (TIOError) e;
+              result.setIoIsSet(true);
+              msg = result;
+            } else 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, listNamespaces_args args, org.apache.thrift.async.AsyncMethodCallback<java.util.List<java.lang.String>> resultHandler) throws org.apache.thrift.TException {
+        iface.listNamespaces(resultHandler);
+      }
+    }
+
   }
 
   public static class exists_args implements org.apache.thrift.TBase<exists_args, exists_args._Fields>, java.io.Serializable, Cloneable, Comparable<exists_args>   {
@@ -36231,13 +36389,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();
@@ -36247,14 +37151,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
@@ -36267,7 +37171,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;
     }
@@ -36327,12 +37231,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)
@@ -36362,7 +37266,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());
       }
@@ -36397,7 +37301,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:");
@@ -36432,15 +37336,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)
@@ -36470,7 +37374,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);
@@ -36485,16 +37389,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()) {
@@ -36507,7 +37411,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)) {
@@ -36523,25 +37427,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>();
 
@@ -36559,6 +37472,8 @@ public class THBaseService {
         switch(fieldId) {
           case 1: // TABLE_NAME
             return TABLE_NAME;
+          case 2: // PRESERVE_SPLITS
+            return PRESERVE_SPLITS;
           default:
             return null;
         }
@@ -36600,45 +37515,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() {
@@ -36646,9 +37572,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;
     }
@@ -36668,6 +37594,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:
@@ -36678,6 +37633,14 @@ public class THBaseService {
         }
         break;
 
+      case PRESERVE_SPLITS:
+        if (value == null) {
+          unsetPreserveSplits();
+        } else {
+          setPreserveSplits((java.lang.Boolean)value);
+        }
+        break;
+
       }
     }
 
@@ -36687,6 +37650,9 @@ public class THBaseService {
       case TABLE_NAME:
         return getTableName();
 
+      case PRESERVE_SPLITS:
+        return isPreserveSplits();
+
       }
       throw new java.lang.IllegalStateException();
     }
@@ -36700,6 +37666,8 @@ public class THBaseService {
       switch (field) {
       case TABLE_NAME:
         return isSetTableName();
+      case PRESERVE_SPLITS:
+        return isSetPreserveSplits();
       }
       throw new java.lang.IllegalStateException();
     }
@@ -36708,12 +37676,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)
@@ -36728,6 +37696,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;
     }
 
@@ -36739,11 +37716,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());
       }
@@ -36760,6 +37739,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;
     }
 
@@ -36778,7 +37767,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:");
@@ -36788,6 +37777,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();
     }
@@ -36797,6 +37790,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();
@@ -36813,21 +37807,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)
@@ -36846,6 +37842,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);
           }
@@ -36854,10 +37858,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);
@@ -36866,32 +37873,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);
       }
     }
 
@@ -36900,13 +37913,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
 
@@ -36977,13 +37990,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();
@@ -36993,14 +38006,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
@@ -37013,7 +38026,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;
     }
@@ -37073,12 +38086,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)
@@ -37108,7 +38121,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());
       }
@@ -37143,7 +38156,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:");
@@ -37178,15 +38191,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)
@@ -37216,7 +38229,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);
@@ -37231,16 +38244,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()) {
@@ -37253,7 +38266,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)) {
@@ -37269,34 +38282,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>();
 
@@ -37314,8 +38318,6 @@ public class THBaseService {
         switch(fieldId) {
           case 1: // TABLE_NAME
             return TABLE_NAME;
-          case 2: // PRESERVE_SPLITS
-            return PRESERVE_SPLITS;
           default:
             return null;
         }
@@ -37357,56 +38359,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() {
@@ -37414,9 +38405,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;
     }
@@ -37436,35 +38427,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:
@@ -37475,14 +38437,6 @@ public class THBaseService {
         }
         break;
 
-      case PRESERVE_SPLITS:
-        if (value == null) {
-          unsetPreserveSplits();
-        } else {
-          setPreserveSplits((java.lang.Boolean)value);
-        }
-        break;
-
       }
     }
 
@@ -37492,9 +38446,6 @@ public class THBaseService {
       case TABLE_NAME:
         return getTableName();
 
-      case PRESERVE_SPLITS:
-        return isPreserveSplits();
-
       }
       throw new java.lang.IllegalStateException();
     }
@@ -37508,8 +38459,6 @@ public class THBaseService {
       switch (field) {
       case TABLE_NAME:
         return isSetTableName();
-      case PRESERVE_SPLITS:
-        return isSetPreserveSplits();
       }
       throw new java.lang.IllegalStateException();
     }
@@ -37518,12 +38467,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)
@@ -37538,15 +38487,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;
     }
 
@@ -37558,13 +38498,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());
       }
@@ -37581,16 +38519,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;
     }
 
@@ -37609,7 +38537,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:");
@@ -37619,10 +38547,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();
     }
@@ -37632,7 +38556,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();
@@ -37649,23 +38572,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)
@@ -37684,14 +38605,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);
           }
@@ -37700,13 +38613,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);
@@ -37715,38 +38625,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);
       }
     }
 
@@ -37755,13 +38659,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
 
@@ -37832,13 +38736,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();
@@ -37848,14 +38752,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
@@ -37868,7 +38772,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;
     }
@@ -37928,12 +38832,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)
@@ -37963,7 +38867,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());
       }
@@ -37998,7 +38902,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:");
@@ -38033,15 +38937,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)
@@ -38071,7 +38975,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);
@@ -38086,16 +38990,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()) {
@@ -38108,7 +39012,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)) {
@@ -38124,23 +39028,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");
 
@@ -38207,13 +39111,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();
@@ -38223,14 +39127,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
@@ -38239,7 +39143,7 @@ public class THBaseService {
     }
 
     /**
-     * the tablename to enable
+     * the tablename to disable
      */
     @org.apache.thrift.annotation.Nullable
     public TTableName getTableName() {
@@ -38247,9 +39151,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;
     }
@@ -38309,12 +39213,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)
@@ -38344,7 +39248,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());
       }
@@ -38379,7 +39283,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:");
@@ -38420,15 +39324,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)
@@ -38458,7 +39362,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);
@@ -38473,22 +39377,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);
@@ -38501,13 +39405,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
 
@@ -38578,13 +39482,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();
@@ -38594,14 +39498,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
@@ -38614,7 +39518,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;
     }
@@ -38674,12 +39578,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)
@@ -38709,7 +39613,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());
       }
@@ -38744,7 +39648,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:");
@@ -38779,15 +39683,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)
@@ -38817,7 +39721,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);
@@ -38832,16 +39736,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()) {
@@ -38854,7 +39758,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)) {
@@ -38870,23 +39774,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");
 
@@ -38953,13 +39857,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();
@@ -38969,14 +39873,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
@@ -38985,7 +39889,7 @@ public class THBaseService {
     }
 
     /**
-     * the tablename to disable
+     * the tablename to check
      */
     @org.apache.thrift.annotation.Nullable
     public TTableName getTableName() {
@@ -38993,9 +39897,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;
     }
@@ -39055,12 +39959,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)
@@ -39090,7 +39994,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());
       }
@@ -39125,7 +40029,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:");
@@ -39166,15 +40070,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)
@@ -39204,7 +40108,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);
@@ -39219,22 +40123,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);
@@ -39247,18 +40151,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>();
@@ -39275,6 +40182,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:
@@ -39318,49 +40227,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;
     }
@@ -39382,6 +40325,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();
@@ -39396,6 +40347,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();
 
@@ -39410,6 +40364,8 @@ public class THBaseService {
       }
 
       switch (field) {
+      case SUCCESS:
+        return isSetSuccess();
       case IO:
         return isSetIo();
       }
@@ -39420,17 +40376,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) {
@@ -39447,6 +40412,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();
@@ -39455,13 +40422,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;
@@ -39490,9 +40467,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");
@@ -39519,21 +40500,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)
@@ -39543,6 +40526,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();
@@ -39563,10 +40554,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);
@@ -39578,32 +40574,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);
@@ -39616,13 +40622,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
@@ -39699,13 +40705,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();
@@ -39715,14 +40721,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
@@ -39741,7 +40747,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;
     }
@@ -39801,12 +40807,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)
@@ -39836,7 +40842,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());
       }
@@ -39871,7 +40877,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:");
@@ -39912,15 +40918,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)
@@ -39950,7 +40956,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);
@@ -39965,22 +40971,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);
@@ -39993,14 +40999,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
@@ -40079,13 +41085,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)
     {
@@ -40098,7 +41104,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()) {
@@ -40106,8 +41112,8 @@ public class THBaseService {
       }
     }
 
-    public isTableEnabled_result deepCopy() {
-      return new isTableEnabled_result(this);
+    public isTableDisabled_result deepCopy() {
+      return new isTableDisabled_result(this);
     }
 
     @Override
@@ -40121,7 +41127,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;
@@ -40145,7 +41151,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;
     }
@@ -40218,12 +41224,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)
@@ -40264,7 +41270,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());
       }
@@ -40309,7 +41315,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:");
@@ -40350,15 +41356,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)
@@ -40396,7 +41402,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);
@@ -40416,16 +41422,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()) {
@@ -40444,7 +41450,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)) {
@@ -40464,13 +41470,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
@@ -40547,13 +41553,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();
@@ -40563,14 +41569,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
@@ -40589,7 +41595,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;
     }
@@ -40649,12 +41655,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)
@@ -40684,7 +41690,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());
       }
@@ -40719,7 +41725,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:");
@@ -40760,15 +41766,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)
@@ -40798,7 +41804,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);
@@ -40813,22 +41819,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);
@@ -40841,14 +41847,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
@@ -40927,13 +41933,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)
     {
@@ -40946,7 +41952,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()) {
@@ -40954,8 +41960,8 @@ public class THBaseService {
       }
     }
 
-    public isTableDisabled_result deepCopy() {
-      return new isTableDisabled_result(this);
+    public isTableAvailable_result deepCopy() {
+      return new isTableAvailable_result(this);
     }
 
     @Override
@@ -40969,7 +41975,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;
@@ -40993,7 +41999,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;
     }
@@ -41066,12 +42072,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)
@@ -41112,7 +42118,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());
       }
@@ -41157,7 +42163,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:");
@@ -41198,15 +42204,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)
@@ -41244,7 +42250,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);
@@ -41264,16 +42270,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()) {
@@ -41292,7 +42298,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)) {
@@ -41312,25 +42318,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>();
 
@@ -41348,6 +42363,8 @@ public class THBaseService {
         switch(fieldId) {
           case 1: // TABLE_NAME
             return TABLE_NAME;
+          case 2: // SPLIT_KEYS
+            return SPLIT_KEYS;
           default:
             return null;
         }
@@ -41394,36 +42411,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;
     }
 
     /**
@@ -41437,7 +42464,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;
     }
@@ -41457,6 +42484,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:
@@ -41467,6 +42541,14 @@ public class THBaseService {
         }
         break;
 
+      case SPLIT_KEYS:
+        if (value == null) {
+          unsetSplitKeys();
+        } else {
+          setSplitKeys((java.util.List<java.nio.ByteBuffer>)value);
+        }
+        break;
+
       }
     }
 
@@ -41476,6 +42558,9 @@ public class THBaseService {
       case TABLE_NAME:
         return getTableName();
 
+      case SPLIT_KEYS:
+        return getSplitKeys();
+
       }
       throw new java.lang.IllegalStateException();
     }
@@ -41489,6 +42574,8 @@ public class THBaseService {
       switch (field) {
       case TABLE_NAME:
         return isSetTableName();
+      case SPLIT_KEYS:
+        return isSetSplitKeys();
       }
       throw new java.lang.IllegalStateException();
     }
@@ -41497,12 +42584,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)
@@ -41517,6 +42604,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;
     }
 
@@ -41528,11 +42624,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());
       }
@@ -41549,6 +42649,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;
     }
 
@@ -41567,7 +42677,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:");
@@ -41577,6 +42687,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();
     }
@@ -41608,15 +42726,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)
@@ -41635,6 +42753,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);
           }
@@ -41646,7 +42782,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);
@@ -41655,32 +42791,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);
+        }
       }
     }
 
@@ -41689,14 +42865,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
@@ -41775,13 +42951,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)
     {
@@ -41794,7 +42970,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()) {
@@ -41802,8 +42978,8 @@ public class THBaseService {
       }
     }
 
-    public isTableAvailable_result deepCopy() {
-      return new isTableAvailable_result(this);
+    public isTableAvailableWithSplit_result deepCopy() {
+      return new isTableAvailableWithSplit_result(this);
     }
 
     @Override
@@ -41817,7 +42993,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;
@@ -41841,7 +43017,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;
     }
@@ -41914,12 +43090,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)
@@ -41960,7 +43136,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());
       }
@@ -42005,7 +43181,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:");
@@ -42046,15 +43222,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)
@@ -42092,7 +43268,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);
@@ -42112,16 +43288,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()) {
@@ -42140,7 +43316,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)) {
@@ -42160,34 +43336,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>();
 
@@ -42205,8 +43381,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;
         }
@@ -42253,50 +43429,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() {
@@ -42304,9 +43478,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;
     }
@@ -42326,50 +43500,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;
       }
     }
 
@@ -42383,11 +43541,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;
 
@@ -42400,8 +43558,8 @@ public class THBaseService {
       case TABLE_NAME:
         return getTableName();
 
-      case SPLIT_KEYS:
-        return getSplitKeys();
+      case COLUMN:
+        return getColumn();
 
       }
       throw new java.lang.IllegalStateException();
@@ -42416,8 +43574,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();
     }
@@ -42426,12 +43584,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)
@@ -42446,12 +43604,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;
       }
 
@@ -42466,15 +43624,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());
       }
@@ -42491,12 +43649,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;
         }
@@ -42519,7 +43677,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:");
@@ -42530,11 +43688,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(")");
@@ -42546,10 +43704,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 {
@@ -42568,15 +43732,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)
@@ -42595,20 +43759,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);
               }
@@ -42624,7 +43779,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);
@@ -42633,16 +43788,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();
@@ -42651,54 +43799,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);
       }
     }
 
@@ -42707,21 +43831,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>();
@@ -42738,8 +43859,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:
@@ -42783,83 +43902,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;
     }
@@ -42881,14 +43966,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();
@@ -42903,9 +43980,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();
 
@@ -42920,8 +43994,6 @@ public class THBaseService {
       }
 
       switch (field) {
-      case SUCCESS:
-        return isSetSuccess();
       case IO:
         return isSetIo();
       }
@@ -42932,26 +44004,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) {
@@ -42968,8 +44031,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();
@@ -42978,23 +44039,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;
@@ -43023,13 +44074,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");
@@ -43056,23 +44103,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)
@@ -43082,14 +44127,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();
@@ -43110,15 +44147,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);
@@ -43130,42 +44162,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);
@@ -43178,32 +44200,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");
 
@@ -43272,37 +44294,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
@@ -43312,7 +44334,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() {
@@ -43320,9 +44342,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;
     }
@@ -43343,18 +44365,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;
     }
 
@@ -43387,7 +44418,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;
 
@@ -43426,12 +44461,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)
@@ -43474,7 +44509,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());
       }
@@ -43519,7 +44554,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:");
@@ -43534,7 +44569,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(")");
@@ -43553,9 +44588,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 {
@@ -43574,15 +44606,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)
@@ -43602,9 +44634,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);
@@ -43621,7 +44652,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);
@@ -43632,7 +44663,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();
@@ -43641,29 +44672,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);
       }
     }
@@ -43673,13 +44703,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
 
@@ -43750,13 +44780,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();
@@ -43766,14 +44796,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
@@ -43786,7 +44816,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;
     }
@@ -43846,12 +44876,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)
@@ -43881,7 +44911,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());
       }
@@ -43916,7 +44946,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:");
@@ -43951,15 +44981,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)
@@ -43989,7 +45019,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);
@@ -44004,16 +45034,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()) {
@@ -44026,7 +45056,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)) {
@@ -44042,32 +45072,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");
 
@@ -44136,37 +45166,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
@@ -44176,7 +45206,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() {
@@ -44184,9 +45214,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;
     }
@@ -44207,27 +45237,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;
     }
 
@@ -44260,11 +45281,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;
 
@@ -44303,12 +45320,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)
@@ -44351,7 +45368,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());
       }
@@ -44396,7 +45413,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:");
@@ -44411,7 +45428,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(")");
@@ -44430,6 +45447,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 {
@@ -44448,15 +45468,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)
@@ -44476,8 +45496,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);
@@ -44494,7 +45515,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);
@@ -44505,7 +45526,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();
@@ -44514,28 +45535,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);
       }
     }
@@ -44545,13 +45567,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
 
@@ -44622,13 +45644,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();
@@ -44638,14 +45660,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
@@ -44658,7 +45680,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;
     }
@@ -44718,12 +45740,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)
@@ -44753,7 +45775,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());
       }
@@ -44788,7 +45810,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:");
@@ -44823,15 +45845,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)
@@ -44861,7 +45883,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);
@@ -44876,16 +45898,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()) {
@@ -44898,7 +45920,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)) {
@@ -44914,34 +45936,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>();
 
@@ -44957,10 +45970,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;
         }
@@ -45005,125 +46016,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;
 
@@ -45133,11 +46097,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();
@@ -45150,10 +46111,8 @@ public class THBaseService {
       }
 
       switch (field) {
-      case TABLE_NAME:
-        return isSetTableName();
-      case COLUMN:
-        return isSetColumn();
+      case DESC:
+        return isSetDesc();
       }
       throw new java.lang.IllegalStateException();
     }
@@ -45162,32 +46121,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;
       }
 
@@ -45198,41 +46148,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;
         }
@@ -45255,22 +46191,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(")");
@@ -45279,18 +46207,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();
       }
     }
 
@@ -45310,15 +46232,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)
@@ -45328,20 +46250,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);
               }
@@ -45357,18 +46270,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();
@@ -45377,30 +46285,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);
       }
     }
 
@@ -45409,13 +46313,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
 
@@ -45486,13 +46390,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();
@@ -45502,14 +46406,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
@@ -45522,7 +46426,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;
     }
@@ -45582,12 +46486,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)
@@ -45617,7 +46521,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());
       }
@@ -45652,7 +46556,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:");
@@ -45687,15 +46591,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)
@@ -45725,7 +46629,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);
@@ -45740,16 +46644,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()) {
@@ -45762,7 +46666,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)) {
@@ -45778,25 +46682,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>();
 
@@ -45812,8 +46716,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;
         }
@@ -45858,78 +46762,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;
 
@@ -45939,8 +46843,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();
@@ -45953,8 +46857,8 @@ public class THBaseService {
       }
 
       switch (field) {
-      case DESC:
-        return isSetDesc();
+      case NAMESPACE_DESC:
+        return isSetNamespaceDesc();
       }
       throw new java.lang.IllegalStateException();
     }
@@ -45963,23 +46867,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;
       }
 
@@ -45990,27 +46894,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;
         }
@@ -46033,14 +46937,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(")");
@@ -46049,12 +46953,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();
       }
     }
 
@@ -46074,15 +46978,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)
@@ -46092,11 +46996,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);
               }
@@ -46112,13 +47016,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();
@@ -46127,26 +47031,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);
       }
     }
 
@@ -46155,13 +47059,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
 
@@ -46232,13 +47136,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();
@@ -46248,14 +47152,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
@@ -46268,7 +47172,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;
     }
@@ -46328,12 +47232,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)
@@ -46363,7 +47267,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());
       }
@@ -46398,7 +47302,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:");
@@ -46433,15 +47337,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)
@@ -46471,7 +47375,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);
@@ -46486,16 +47390,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()) {
@@ -46508,7 +47412,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)) {
@@ -46524,13 +47428,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
@@ -46607,13 +47511,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();
@@ -46623,14 +47527,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
@@ -46649,7 +47553,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;
     }
@@ -46709,12 +47613,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)
@@ -46744,7 +47648,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());
       }
@@ -46779,7 +47683,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:");
@@ -46820,15 +47724,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)
@@ -46858,7 +47762,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);
@@ -46873,22 +47777,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);
@@ -46901,13 +47805,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
 
@@ -46978,13 +47882,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();
@@ -46994,14 +47898,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
@@ -47014,7 +47918,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;
     }
@@ -47074,12 +47978,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)
@@ -47109,7 +48013,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());
       }
@@ -47144,7 +48048,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:");
@@ -47179,15 +48083,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)
@@ -47217,7 +48121,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);
@@ -47232,16 +48136,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()) {
@@ -47254,7 +48158,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)) {
@@ -47270,25 +48174,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>();
 
@@ -47304,8 +48208,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;
         }
@@ -47350,78 +48254,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;
 
@@ -47431,8 +48335,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();
@@ -47445,8 +48349,8 @@ public class THBaseService {
       }
 
       switch (field) {
-      case NAMESPACE_DESC:
-        return isSetNamespaceDesc();
+      case NAME:
+        return isSetName();
       }
       throw new java.lang.IllegalStateException();
     }
@@ -47455,23 +48359,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;
       }
 
@@ -47482,27 +48386,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;
         }
@@ -47525,14 +48429,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(")");
@@ -47541,13 +48445,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 {
@@ -47566,15 +48467,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)
@@ -47584,11 +48485,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);
               }
@@ -47604,13 +48504,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();
@@ -47619,26 +48519,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);
       }
     }
 
@@ -47647,13 +48546,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
 
@@ -47724,13 +48623,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();
@@ -47740,14 +48639,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
@@ -47760,7 +48659,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;
     }
@@ -47820,12 +48719,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)
@@ -47855,7 +48754,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());
       }
@@ -47890,7 +48789,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:");
@@ -47925,15 +48824,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)
@@ -47963,7 +48862,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);
@@ -47978,16 +48877,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()) {
@@ -48000,7 +48899,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)) {
@@ -48016,23 +48915,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");
 
@@ -48099,13 +48998,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();
@@ -48115,14 +49014,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
@@ -48131,7 +49030,7 @@ public class THBaseService {
     }
 
     /**
-     * namespace name
+     * name of namespace descriptor
      */
     @org.apache.thrift.annotation.Nullable
     public java.lang.String getName() {
@@ -48139,9 +49038,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;
     }
@@ -48201,12 +49100,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)
@@ -48236,7 +49135,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());
       }
@@ -48271,7 +49170,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:");
@@ -48309,15 +49208,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)
@@ -48346,7 +49245,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);
@@ -48361,22 +49260,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);
@@ -48388,18 +49287,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>();
@@ -48416,6 +49318,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:
@@ -48462,46 +49366,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;
     }
@@ -48523,6 +49460,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((TNamespaceDescriptor)value);
+        }
+        break;
+
       case IO:
         if (value == null) {
           unsetIo();
@@ -48537,6 +49482,9 @@ public class THBaseService {
     @org.apache.thrift.annotation.Nullable
     public java.lang.Object getFieldValue(_Fields field) {
       switch (field) {
+      case SUCCESS:
+        return getSuccess();
+
       case IO:
         return getIo();
 
@@ -48551,6 +49499,8 @@ public class THBaseService {
       }
 
       switch (field) {
+      case SUCCESS:
+        return isSetSuccess();
       case IO:
         return isSetIo();
       }
@@ -48561,17 +49511,26 @@ public class THBaseService {
     public boolean equals(java.lang.Object that) {
       if (that == null)
         return false;
-      if (that instanceof deleteNamespace_result)
-        return this.equals((deleteNamespace_result)that);
+      if (that instanceof getNamespaceDescriptor_result)
+        return this.equals((getNamespaceDescriptor_result)that);
       return false;
     }
 
-    public boolean equals(deleteNamespace_result that) {
+    public boolean equals(getNamespaceDescriptor_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;
+      }
+
       boolean this_present_io = true && this.isSetIo();
       boolean that_present_io = true && that.isSetIo();
       if (this_present_io || that_present_io) {
@@ -48588,6 +49547,10 @@ public class THBaseService {
     public int hashCode() {
       int hashCode = 1;
 
+      hashCode = hashCode * 8191 + ((isSetSuccess()) ? 131071 : 524287);
+      if (isSetSuccess())
+        hashCode = hashCode * 8191 + success.hashCode();
+
       hashCode = hashCode * 8191 + ((isSetIo()) ? 131071 : 524287);
       if (isSetIo())
         hashCode = hashCode * 8191 + io.hashCode();
@@ -48596,13 +49559,23 @@ public class THBaseService {
     }
 
     @Override
-    public int compareTo(deleteNamespace_result other) {
+    public int compareTo(getNamespaceDescriptor_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;
@@ -48631,9 +49604,17 @@ public class THBaseService {
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("deleteNamespace_result(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("getNamespaceDescriptor_result(");
       boolean first = true;
 
+      sb.append("success:");
+      if (this.success == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.success);
+      }
+      first = false;
+      if (!first) sb.append(", ");
       sb.append("io:");
       if (this.io == null) {
         sb.append("null");
@@ -48648,6 +49629,9 @@ public class THBaseService {
     public void validate() throws org.apache.thrift.TException {
       // check for required fields
       // check for sub-struct validity
+      if (success != null) {
+        success.validate();
+      }
     }
 
     private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
@@ -48666,15 +49650,15 @@ public class THBaseService {
       }
     }
 
-    private static class deleteNamespace_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public deleteNamespace_resultStandardScheme getScheme() {
-        return new deleteNamespace_resultStandardScheme();
+    private static class getNamespaceDescriptor_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getNamespaceDescriptor_resultStandardScheme getScheme() {
+        return new getNamespaceDescriptor_resultStandardScheme();
       }
     }
 
-    private static class deleteNamespace_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<deleteNamespace_result> {
+    private static class getNamespaceDescriptor_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<getNamespaceDescriptor_result> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, deleteNamespace_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, getNamespaceDescriptor_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -48684,6 +49668,15 @@ public class THBaseService {
             break;
           }
           switch (schemeField.id) {
+            case 0: // SUCCESS
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.success = new TNamespaceDescriptor();
+                struct.success.read(iprot);
+                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();
@@ -48704,10 +49697,15 @@ public class THBaseService {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, deleteNamespace_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, getNamespaceDescriptor_result struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.success != null) {
+          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+          struct.success.write(oprot);
+          oprot.writeFieldEnd();
+        }
         if (struct.io != null) {
           oprot.writeFieldBegin(IO_FIELD_DESC);
           struct.io.write(oprot);
@@ -48719,32 +49717,43 @@ public class THBaseService {
 
     }
 
-    private static class deleteNamespace_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public deleteNamespace_resultTupleScheme getScheme() {
-        return new deleteNamespace_resultTupleScheme();
+    private static class getNamespaceDescriptor_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getNamespaceDescriptor_resultTupleScheme getScheme() {
+        return new getNamespaceDescriptor_resultTupleScheme();
       }
     }
 
-    private static class deleteNamespace_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<deleteNamespace_result> {
+    private static class getNamespaceDescriptor_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<getNamespaceDescriptor_result> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, deleteNamespace_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, getNamespaceDescriptor_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()) {
+          struct.success.write(oprot);
+        }
         if (struct.isSetIo()) {
           struct.io.write(oprot);
         }
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, deleteNamespace_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, getNamespaceDescriptor_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 = new TNamespaceDescriptor();
+          struct.success.read(iprot);
+          struct.setSuccessIsSet(true);
+        }
+        if (incoming.get(1)) {
           struct.io = new TIOError();
           struct.io.read(iprot);
           struct.setIoIsSet(true);
@@ -48757,25 +49766,17 @@ public class THBaseService {
     }
   }
 
-  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");
+  public static class listNamespaceDescriptors_args implements org.apache.thrift.TBase<listNamespaceDescriptors_args, listNamespaceDescriptors_args._Fields>, java.io.Serializable, Cloneable, Comparable<listNamespaceDescriptors_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("listNamespaceDescriptors_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 getNamespaceDescriptor_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getNamespaceDescriptor_argsTupleSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new listNamespaceDescriptors_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new listNamespaceDescriptors_argsTupleSchemeFactory();
 
-    /**
-     * 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 {
-      /**
-       * name of namespace descriptor
-       */
-      NAME((short)1, "name");
+;
 
       private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
 
@@ -48791,8 +49792,6 @@ public class THBaseService {
       @org.apache.thrift.annotation.Nullable
       public static _Fields findByThriftId(int fieldId) {
         switch(fieldId) {
-          case 1: // NAME
-            return NAME;
           default:
             return null;
         }
@@ -48832,95 +49831,38 @@ public class THBaseService {
         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.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(getNamespaceDescriptor_args.class, metaDataMap);
-    }
-
-    public getNamespaceDescriptor_args() {
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(listNamespaceDescriptors_args.class, metaDataMap);
     }
 
-    public getNamespaceDescriptor_args(
-      java.lang.String name)
-    {
-      this();
-      this.name = name;
+    public listNamespaceDescriptors_args() {
     }
 
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public getNamespaceDescriptor_args(getNamespaceDescriptor_args other) {
-      if (other.isSetName()) {
-        this.name = other.name;
-      }
+    public listNamespaceDescriptors_args(listNamespaceDescriptors_args other) {
     }
 
-    public getNamespaceDescriptor_args deepCopy() {
-      return new getNamespaceDescriptor_args(this);
+    public listNamespaceDescriptors_args deepCopy() {
+      return new listNamespaceDescriptors_args(this);
     }
 
     @Override
     public void clear() {
-      this.name = null;
-    }
-
-    /**
-     * name of namespace descriptor
-     */
-    @org.apache.thrift.annotation.Nullable
-    public java.lang.String getName() {
-      return this.name;
-    }
-
-    /**
-     * name of namespace descriptor
-     */
-    public getNamespaceDescriptor_args setName(@org.apache.thrift.annotation.Nullable java.lang.String name) {
-      this.name = name;
-      return this;
-    }
-
-    public void unsetName() {
-      this.name = 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 setNameIsSet(boolean value) {
-      if (!value) {
-        this.name = null;
-      }
     }
 
     public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
       switch (field) {
-      case NAME:
-        if (value == null) {
-          unsetName();
-        } else {
-          setName((java.lang.String)value);
-        }
-        break;
-
       }
     }
 
     @org.apache.thrift.annotation.Nullable
     public java.lang.Object getFieldValue(_Fields field) {
       switch (field) {
-      case NAME:
-        return getName();
-
       }
       throw new java.lang.IllegalStateException();
     }
@@ -48932,8 +49874,6 @@ public class THBaseService {
       }
 
       switch (field) {
-      case NAME:
-        return isSetName();
       }
       throw new java.lang.IllegalStateException();
     }
@@ -48942,26 +49882,17 @@ public class THBaseService {
     public boolean equals(java.lang.Object that) {
       if (that == null)
         return false;
-      if (that instanceof getNamespaceDescriptor_args)
-        return this.equals((getNamespaceDescriptor_args)that);
+      if (that instanceof listNamespaceDescriptors_args)
+        return this.equals((listNamespaceDescriptors_args)that);
       return false;
     }
 
-    public boolean equals(getNamespaceDescriptor_args that) {
+    public boolean equals(listNamespaceDescriptors_args that) {
       if (that == null)
         return false;
       if (this == that)
         return true;
 
-      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.name.equals(that.name))
-          return false;
-      }
-
       return true;
     }
 
@@ -48969,31 +49900,17 @@ public class THBaseService {
     public int hashCode() {
       int hashCode = 1;
 
-      hashCode = hashCode * 8191 + ((isSetName()) ? 131071 : 524287);
-      if (isSetName())
-        hashCode = hashCode * 8191 + name.hashCode();
-
       return hashCode;
     }
 
     @Override
-    public int compareTo(getNamespaceDescriptor_args other) {
+    public int compareTo(listNamespaceDescriptors_args other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
 
       int lastComparison = 0;
 
-      lastComparison = java.lang.Boolean.valueOf(isSetName()).compareTo(other.isSetName());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetName()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.name, other.name);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
       return 0;
     }
 
@@ -49012,25 +49929,15 @@ public class THBaseService {
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("getNamespaceDescriptor_args(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("listNamespaceDescriptors_args(");
       boolean first = true;
 
-      sb.append("name:");
-      if (this.name == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.name);
-      }
-      first = false;
       sb.append(")");
       return sb.toString();
     }
 
     public void validate() throws org.apache.thrift.TException {
       // check for required fields
-      if (name == null) {
-        throw new org.apache.thrift.protocol.TProtocolException("Required field 'name' was not present! Struct: " + toString());
-      }
       // check for sub-struct validity
     }
 
@@ -49050,15 +49957,15 @@ public class THBaseService {
       }
     }
 
-    private static class getNamespaceDescriptor_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public getNamespaceDescriptor_argsStandardScheme getScheme() {
-        return new getNamespaceDescriptor_argsStandardScheme();
+    private static class listNamespaceDescriptors_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public listNamespaceDescriptors_argsStandardScheme getScheme() {
+        return new listNamespaceDescriptors_argsStandardScheme();
       }
     }
 
-    private static class getNamespaceDescriptor_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<getNamespaceDescriptor_args> {
+    private static class listNamespaceDescriptors_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<listNamespaceDescriptors_args> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, getNamespaceDescriptor_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, listNamespaceDescriptors_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -49068,14 +49975,6 @@ public class THBaseService {
             break;
           }
           switch (schemeField.id) {
-            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);
-              }
-              break;
             default:
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
           }
@@ -49087,40 +49986,32 @@ public class THBaseService {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, getNamespaceDescriptor_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, listNamespaceDescriptors_args struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
-        if (struct.name != null) {
-          oprot.writeFieldBegin(NAME_FIELD_DESC);
-          oprot.writeString(struct.name);
-          oprot.writeFieldEnd();
-        }
         oprot.writeFieldStop();
         oprot.writeStructEnd();
       }
 
     }
 
-    private static class getNamespaceDescriptor_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public getNamespaceDescriptor_argsTupleScheme getScheme() {
-        return new getNamespaceDescriptor_argsTupleScheme();
+    private static class listNamespaceDescriptors_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public listNamespaceDescriptors_argsTupleScheme getScheme() {
+        return new listNamespaceDescriptors_argsTupleScheme();
       }
     }
 
-    private static class getNamespaceDescriptor_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<getNamespaceDescriptor_args> {
+    private static class listNamespaceDescriptors_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<listNamespaceDescriptors_args> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, getNamespaceDescriptor_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, listNamespaceDescriptors_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, getNamespaceDescriptor_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, listNamespaceDescriptors_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);
       }
     }
 
@@ -49129,16 +50020,16 @@ public class THBaseService {
     }
   }
 
-  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");
+  public static class listNamespaceDescriptors_result implements org.apache.thrift.TBase<listNamespaceDescriptors_result, listNamespaceDescriptors_result._Fields>, java.io.Serializable, Cloneable, Comparable<listNamespaceDescriptors_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("listNamespaceDescriptors_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 SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.LIST, (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 getNamespaceDescriptor_resultStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getNamespaceDescriptor_resultTupleSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new listNamespaceDescriptors_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new listNamespaceDescriptors_resultTupleSchemeFactory();
 
-    public @org.apache.thrift.annotation.Nullable TNamespaceDescriptor success; // required
+    public @org.apache.thrift.annotation.Nullable java.util.List<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. */
@@ -49209,18 +50100,19 @@ public class THBaseService {
     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)));
+          new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+              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(getNamespaceDescriptor_result.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(listNamespaceDescriptors_result.class, metaDataMap);
     }
 
-    public getNamespaceDescriptor_result() {
+    public listNamespaceDescriptors_result() {
     }
 
-    public getNamespaceDescriptor_result(
-      TNamespaceDescriptor success,
+    public listNamespaceDescriptors_result(
+      java.util.List<TNamespaceDescriptor> success,
       TIOError io)
     {
       this();
@@ -49231,17 +50123,21 @@ public class THBaseService {
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public getNamespaceDescriptor_result(getNamespaceDescriptor_result other) {
+    public listNamespaceDescriptors_result(listNamespaceDescriptors_result other) {
       if (other.isSetSuccess()) {
-        this.success = new TNamespaceDescriptor(other.success);
+        java.util.List<TNamespaceDescriptor> __this__success = new java.util.ArrayList<TNamespaceDescriptor>(other.success.size());
+        for (TNamespaceDescriptor other_element : other.success) {
+          __this__success.add(new TNamespaceDescriptor(other_element));
+        }
+        this.success = __this__success;
       }
       if (other.isSetIo()) {
         this.io = new TIOError(other.io);
       }
     }
 
-    public getNamespaceDescriptor_result deepCopy() {
-      return new getNamespaceDescriptor_result(this);
+    public listNamespaceDescriptors_result deepCopy() {
+      return new listNamespaceDescriptors_result(this);
     }
 
     @Override
@@ -49250,12 +50146,28 @@ public class THBaseService {
       this.io = null;
     }
 
+    public int getSuccessSize() {
+      return (this.success == null) ? 0 : this.success.size();
+    }
+
     @org.apache.thrift.annotation.Nullable
-    public TNamespaceDescriptor getSuccess() {
+    public java.util.Iterator<TNamespaceDescriptor> getSuccessIterator() {
+      return (this.success == null) ? null : this.success.iterator();
+    }
+
+    public void addToSuccess(TNamespaceDescriptor elem) {
+      if (this.success == null) {
+        this.success = new java.util.ArrayList<TNamespaceDescriptor>();
+      }
+      this.success.add(elem);
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.util.List<TNamespaceDescriptor> getSuccess() {
       return this.success;
     }
 
-    public getNamespaceDescriptor_result setSuccess(@org.apache.thrift.annotation.Nullable TNamespaceDescriptor success) {
+    public listNamespaceDescriptors_result setSuccess(@org.apache.thrift.annotation.Nullable java.util.List<TNamespaceDescriptor> success) {
       this.success = success;
       return this;
     }
@@ -49280,7 +50192,7 @@ public class THBaseService {
       return this.io;
     }
 
-    public getNamespaceDescriptor_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) {
+    public listNamespaceDescriptors_result setIo(@org.apache.thrift.annotation.Nullable TIOError io) {
       this.io = io;
       return this;
     }
@@ -49306,7 +50218,7 @@ public class THBaseService {
         if (value == null) {
           unsetSuccess();
         } else {
-          setSuccess((TNamespaceDescriptor)value);
+          setSuccess((java.util.List<TNamespaceDescriptor>)value);
         }
         break;
 
@@ -49353,12 +50265,12 @@ public class THBaseService {
     public boolean equals(java.lang.Object that) {
       if (that == null)
         return false;
-      if (that instanceof getNamespaceDescriptor_result)
-        return this.equals((getNamespaceDescriptor_result)that);
+      if (that instanceof listNamespaceDescriptors_result)
+        return this.equals((listNamespaceDescriptors_result)that);
       return false;
     }
 
-    public boolean equals(getNamespaceDescriptor_result that) {
+    public boolean equals(listNamespaceDescriptors_result that) {
       if (that == null)
         return false;
       if (this == that)
@@ -49401,7 +50313,7 @@ public class THBaseService {
     }
 
     @Override
-    public int compareTo(getNamespaceDescriptor_result other) {
+    public int compareTo(listNamespaceDescriptors_result other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -49446,7 +50358,7 @@ public class THBaseService {
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("getNamespaceDescriptor_result(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("listNamespaceDescriptors_result(");
       boolean first = true;
 
       sb.append("success:");
@@ -49471,9 +50383,6 @@ public class THBaseService {
     public void validate() throws org.apache.thrift.TException {
       // check for required fields
       // check for sub-struct validity
-      if (success != null) {
-        success.validate();
-      }
     }
 
     private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
@@ -49492,15 +50401,15 @@ public class THBaseService {
       }
     }
 
-    private static class getNamespaceDescriptor_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public getNamespaceDescriptor_resultStandardScheme getScheme() {
-        return new getNamespaceDescriptor_resultStandardScheme();
+    private static class listNamespaceDescriptors_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public listNamespaceDescriptors_resultStandardScheme getScheme() {
+        return new listNamespaceDescriptors_resultStandardScheme();
       }
     }
 
-    private static class getNamespaceDescriptor_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<getNamespaceDescriptor_result> {
+    private static class listNamespaceDescriptors_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<listNamespaceDescriptors_result> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, getNamespaceDescriptor_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, listNamespaceDescriptors_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -49511,9 +50420,19 @@ public class THBaseService {
           }
           switch (schemeField.id) {
             case 0: // SUCCESS
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.success = new TNamespaceDescriptor();
-                struct.success.read(iprot);
+              if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+                {
+                  org.apache.thrift.protocol.TList _list334 = iprot.readListBegin();
+                  struct.success = new java.util.ArrayList<TNamespaceDescriptor>(_list334.size);
+                  @org.apache.thrift.annotation.Nullable TNamespaceDescriptor _elem335;
+                  for (int _i336 = 0; _i336 < _list334.size; ++_i336)
+                  {
+                    _elem335 = new TNamespaceDescriptor();
+                    _elem335.read(iprot);
+                    struct.success.add(_elem335);
+                  }
+                  iprot.readListEnd();
+                }
                 struct.setSuccessIsSet(true);
               } else { 
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
@@ -49539,13 +50458,20 @@ public class THBaseService {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, getNamespaceDescriptor_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, listNamespaceDescriptors_result struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
         if (struct.success != null) {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
-          struct.success.write(oprot);
+          {
+            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
+            for (TNamespaceDescriptor _iter337 : struct.success)
+            {
+              _iter337.write(oprot);
+            }
+            oprot.writeListEnd();
+          }
           oprot.writeFieldEnd();
         }
         if (struct.io != null) {
@@ -49559,16 +50485,16 @@ public class THBaseService {
 
     }
 
-    private static class getNamespaceDescriptor_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public getNamespaceDescriptor_resultTupleScheme getScheme() {
-        return new getNamespaceDescriptor_resultTupleScheme();
+    private static class listNamespaceDescriptors_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public listNamespaceDescriptors_resultTupleScheme getScheme() {
+        return new listNamespaceDescriptors_resultTupleScheme();
       }
     }
 
-    private static class getNamespaceDescriptor_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<getNamespaceDescriptor_result> {
+    private static class listNamespaceDescriptors_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<listNamespaceDescriptors_result> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, getNamespaceDescriptor_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, listNamespaceDescriptors_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()) {
@@ -49579,7 +50505,13 @@ public class THBaseService {
         }
         oprot.writeBitSet(optionals, 2);
         if (struct.isSetSuccess()) {
-          struct.success.write(oprot);
+          {
+            oprot.writeI32(struct.success.size());
+            for (TNamespaceDescriptor _iter338 : struct.success)
+            {
+              _iter338.write(oprot);
+            }
+          }
         }
         if (struct.isSetIo()) {
           struct.io.write(oprot);
@@ -49587,12 +50519,21 @@ public class THBaseService {
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, getNamespaceDescriptor_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, listNamespaceDescriptors_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)) {
-          struct.success = new TNamespaceDescriptor();
-          struct.success.read(iprot);
+          {
+            org.apache.thrift.protocol.TList _list339 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new java.util.ArrayList<TNamespaceDescriptor>(_list339.size);
+            @org.apache.thrift.annotation.Nullable TNamespaceDescriptor _elem340;
+            for (int _i341 = 0; _i341 < _list339.size; ++_i341)
+            {
+              _elem340 = new TNamespaceDescriptor();
+              _elem340.read(iprot);
+              struct.success.add(_elem340);
+            }
+          }
           struct.setSuccessIsSet(true);
         }
         if (incoming.get(1)) {
@@ -49608,12 +50549,12 @@ public class THBaseService {
     }
   }
 
-  public static class listNamespaceDescriptors_args implements org.apache.thrift.TBase<listNamespaceDescriptors_args, listNamespaceDescriptors_args._Fields>, java.io.Serializable, Cloneable, Comparable<listNamespaceDescriptors_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("listNamespaceDescriptors_args");
+  public static class listNamespaces_args implements org.apache.thrift.TBase<listNamespaces_args, listNamespaces_args._Fields>, java.io.Serializable, Cloneable, Comparable<listNamespaces_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("listNamespaces_args");
 
 
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new listNamespaceDescriptors_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new listNamespaceDescriptors_argsTupleSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new listNamespaces_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new listNamespaces_argsTupleSchemeFactory();
 
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
@@ -49677,20 +50618,20 @@ public class THBaseService {
     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(listNamespaceDescriptors_args.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(listNamespaces_args.class, metaDataMap);
     }
 
-    public listNamespaceDescriptors_args() {
+    public listNamespaces_args() {
     }
 
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public listNamespaceDescriptors_args(listNamespaceDescriptors_args other) {
+    public listNamespaces_args(listNamespaces_args other) {
     }
 
-    public listNamespaceDescriptors_args deepCopy() {
-      return new listNamespaceDescriptors_args(this);
+    public listNamespaces_args deepCopy() {
+      return new listNamespaces_args(this);
     }
 
     @Override
@@ -49724,12 +50665,12 @@ public class THBaseService {
     public boolean equals(java.lang.Object that) {
       if (that == null)
         return false;
-      if (that instanceof listNamespaceDescriptors_args)
-        return this.equals((listNamespaceDescriptors_args)that);
+      if (that instanceof listNamespaces_args)
+        return this.equals((listNamespaces_args)that);
       return false;
     }
 
-    public boolean equals(listNamespaceDescriptors_args that) {
+    public boolean equals(listNamespaces_args that) {
       if (that == null)
         return false;
       if (this == that)
@@ -49746,7 +50687,7 @@ public class THBaseService {
     }
 
     @Override
-    public int compareTo(listNamespaceDescriptors_args other) {
+    public int compareTo(listNamespaces_args other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -49771,7 +50712,7 @@ public class THBaseService {
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("listNamespaceDescriptors_args(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("listNamespaces_args(");
       boolean first = true;
 
       sb.append(")");
@@ -49799,15 +50740,15 @@ public class THBaseService {
       }
     }
 
-    private static class listNamespaceDescriptors_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public listNamespaceDescriptors_argsStandardScheme getScheme() {
-        return new listNamespaceDescriptors_argsStandardScheme();
+    private static class listNamespaces_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public listNamespaces_argsStandardScheme getScheme() {
+        return new listNamespaces_argsStandardScheme();
       }
     }
 
-    private static class listNamespaceDescriptors_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<listNamespaceDescriptors_args> {
+    private static class listNamespaces_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<listNamespaces_args> {
 
... 335 lines suppressed ...