You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by dl...@apache.org on 2022/09/26 11:23:24 UTC

[accumulo] branch main updated: Added support for setting multiple properties at once atomically (#2799)

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

dlmarion pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/accumulo.git


The following commit(s) were added to refs/heads/main by this push:
     new f020a78f2e Added support for setting multiple properties at once atomically (#2799)
f020a78f2e is described below

commit f020a78f2e1dfab062dad4338293fc60649e7f05
Author: Christopher L. Shannon <ch...@gmail.com>
AuthorDate: Mon Sep 26 07:23:17 2022 -0400

    Added support for setting multiple properties at once atomically (#2799)
    
    Added new API to support setting multiple properties atomically at the
    System, Namespace, and Table properties levels. If the server detects
    that properties have changed during modification, then a
    ConcurrentModificationException will be thrown.
    
    Closes #2692
    
    
    Co-authored-by: Emily Tchai <71...@users.noreply.github.com>
---
 .../core/client/admin/InstanceOperations.java      |   32 +
 .../core/client/admin/NamespaceOperations.java     |   23 +
 .../core/client/admin/TableOperations.java         |   38 +
 .../core/clientImpl/InstanceOperationsImpl.java    |   39 +
 .../core/clientImpl/NamespaceOperationsImpl.java   |   55 +
 .../core/clientImpl/TableOperationsImpl.java       |   55 +
 .../core/rpc/clients/ManagerThriftClient.java      |    7 +
 .../core/clientImpl/thrift/ClientService.java      | 8545 ++++++++++++++++++--
 .../clientImpl/thrift/TVersionedProperties.java    |  545 ++
 .../ThriftConcurrentModificationException.java     |  393 +
 .../core/manager/thrift/ManagerClientService.java  | 6659 ++++++++++++---
 core/src/main/thrift/client.thrift                 |   51 +
 core/src/main/thrift/manager.thrift                |   36 +-
 .../core/clientImpl/TableOperationsHelperTest.java |   19 +
 .../server/client/ClientServiceHandler.java        |   57 +
 .../server/conf/codec/VersionedProperties.java     |   13 +
 .../accumulo/server/conf/store/PropStore.java      |   20 +-
 .../server/conf/store/impl/ZooPropStore.java       |   55 +
 .../org/apache/accumulo/server/util/PropUtil.java  |   23 +-
 .../accumulo/server/util/SystemPropUtil.java       |   46 +-
 .../manager/ManagerClientServiceHandler.java       |   86 +
 .../accumulo/test/DeprecatedPropertyUtilIT.java    |   48 +
 .../org/apache/accumulo/test/NamespacesIT.java     |   30 +
 .../accumulo/test/conf/PropStoreConfigIT.java      |  166 +
 .../accumulo/test/functional/ManagerApiIT.java     |   43 +-
 .../accumulo/test/functional/PermissionsIT.java    |   37 +
 26 files changed, 15268 insertions(+), 1853 deletions(-)

diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperations.java b/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperations.java
index 347dff76ac..88ccabc53e 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperations.java
@@ -18,12 +18,15 @@
  */
 package org.apache.accumulo.core.client.admin;
 
+import java.util.ConcurrentModificationException;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.function.Consumer;
 
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties;
 import org.apache.accumulo.core.data.InstanceId;
 
 public interface InstanceOperations {
@@ -48,6 +51,27 @@ public interface InstanceOperations {
   void setProperty(final String property, final String value)
       throws AccumuloException, AccumuloSecurityException;
 
+  /**
+   * Modify system properties using a Consumer that accepts a mutable map containing the current
+   * system property overrides stored in ZooKeeper. If the supplied Consumer alters the map without
+   * throwing an Exception, then the resulting map will atomically replace the current system
+   * property overrides in ZooKeeper. Only properties which can be stored in ZooKeeper will be
+   * accepted.
+   *
+   * @throws AccumuloException
+   *           if a general error occurs
+   * @throws AccumuloSecurityException
+   *           if the user does not have permission
+   * @throws IllegalArgumentException
+   *           if the Consumer alters the map by adding properties that cannot be stored in
+   *           ZooKeeper
+   * @throws ConcurrentModificationException
+   *           without altering the stored properties if the server reports that the properties have
+   *           been changed by another process
+   */
+  void modifyProperties(Consumer<Map<String,String>> mapMutator) throws AccumuloException,
+      AccumuloSecurityException, IllegalArgumentException, ConcurrentModificationException;
+
   /**
    * Removes a system property from zookeeper. Changes can be seen using
    * {@link #getSystemConfiguration()}
@@ -70,6 +94,14 @@ public interface InstanceOperations {
    */
   Map<String,String> getSystemConfiguration() throws AccumuloException, AccumuloSecurityException;
 
+  /**
+   * Retrieve the configured System properties from zookeeper
+   *
+   * @return {@link TVersionedProperties} containing a map of system properties set in zookeeper
+   *         that can be changed as well as the version of those properties.
+   */
+  TVersionedProperties getSystemProperties() throws AccumuloException, AccumuloSecurityException;
+
   /**
    * Retrieve the site configuration (that is set in the server configuration file).
    *
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperations.java b/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperations.java
index 740885fbfb..c7c9f7c0e9 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperations.java
@@ -22,6 +22,7 @@ import java.util.EnumSet;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.SortedSet;
+import java.util.function.Consumer;
 
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -169,6 +170,9 @@ public interface NamespaceOperations {
   void setProperty(String namespace, String property, String value)
       throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException;
 
+  void modifyProperties(String namespace, Consumer<Map<String,String>> mapMutator)
+      throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException;
+
   /**
    * Removes a property from a namespace. Note that it may take a few seconds to propagate the
    * change everywhere.
@@ -230,6 +234,25 @@ public interface NamespaceOperations {
   Map<String,String> getConfiguration(String namespace)
       throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException;
 
+  /**
+   * Gets properties specific to this namespace. Note that recently changed properties may not be
+   * available immediately. This new method returns a Map instead of an Iterable.
+   *
+   * @param namespace
+   *          the name of the namespace
+   * @return per-table properties specific to this namespace. Note that recently changed properties
+   *         may not be visible immediately.
+   * @throws AccumuloException
+   *           if a general error occurs
+   * @throws AccumuloSecurityException
+   *           if the user does not have permission
+   * @throws NamespaceNotFoundException
+   *           if the specified namespace doesn't exist
+   * @since 2.1.0
+   */
+  Map<String,String> getNamespaceProperties(String namespace)
+      throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException;
+
   /**
    * Get a mapping of namespace name to internal namespace id.
    *
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java
index 3ce87e2886..7521977779 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java
@@ -20,6 +20,7 @@ package org.apache.accumulo.core.client.admin;
 
 import java.io.IOException;
 import java.util.Collection;
+import java.util.ConcurrentModificationException;
 import java.util.EnumSet;
 import java.util.List;
 import java.util.Map;
@@ -27,6 +28,7 @@ import java.util.Map.Entry;
 import java.util.Set;
 import java.util.SortedSet;
 import java.util.concurrent.Executor;
+import java.util.function.Consumer;
 import java.util.function.Predicate;
 
 import org.apache.accumulo.core.client.AccumuloException;
@@ -604,6 +606,25 @@ public interface TableOperations {
   void setProperty(String tableName, String property, String value)
       throws AccumuloException, AccumuloSecurityException;
 
+  /**
+   * Modify table properties using a Consumer that accepts a mutable map containing the current
+   * table properties. If the supplied Consumer alters the map without throwing an Exception, then
+   * the resulting map will atomically replace the current table properties.
+   *
+   * @throws AccumuloException
+   *           if a general error occurs
+   * @throws AccumuloSecurityException
+   *           if the user does not have permission
+   * @throws IllegalArgumentException
+   *           if the Consumer alters the map by adding properties that cannot be stored
+   * @throws ConcurrentModificationException
+   *           without altering the stored properties if the server reports that the properties have
+   *           been changed by another process
+   */
+  void modifyProperties(String tableName, Consumer<Map<String,String>> mapMutator)
+      throws AccumuloException, AccumuloSecurityException, IllegalArgumentException,
+      ConcurrentModificationException;
+
   /**
    * Removes a property from a table. This operation is asynchronous and eventually consistent. Not
    * all tablets in a table will acknowledge this altered value immediately nor at the same time.
@@ -659,6 +680,23 @@ public interface TableOperations {
   Map<String,String> getConfiguration(String tableName)
       throws AccumuloException, TableNotFoundException;
 
+  /**
+   * Gets per-table properties of a table. This operation is asynchronous and eventually consistent.
+   * It is not guaranteed that all tablets in a table will return the same values. Within a few
+   * seconds without another change, all tablets in a table should be consistent. The clone table
+   * feature can be used if consistency is required.
+   *
+   * @param tableName
+   *          the name of the table
+   * @return per-table properties visible by this table. Note that recently changed properties may
+   *         not be visible immediately.
+   * @throws TableNotFoundException
+   *           if the table does not exist
+   * @since 2.1.0
+   */
+  Map<String,String> getTableProperties(String tableName)
+      throws AccumuloException, TableNotFoundException;
+
   /**
    * Sets a table's locality groups. A table's locality groups can be changed at any time.
    *
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/InstanceOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/InstanceOperationsImpl.java
index ce01a96ae9..995942dcbf 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/InstanceOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/InstanceOperationsImpl.java
@@ -28,12 +28,15 @@ import static org.apache.accumulo.core.rpc.ThriftUtil.returnClient;
 
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.ConcurrentModificationException;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
 import java.util.Set;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
+import java.util.function.Consumer;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloException;
@@ -44,6 +47,7 @@ import org.apache.accumulo.core.client.admin.ActiveCompaction.CompactionHost;
 import org.apache.accumulo.core.client.admin.ActiveScan;
 import org.apache.accumulo.core.client.admin.InstanceOperations;
 import org.apache.accumulo.core.clientImpl.thrift.ConfigurationType;
+import org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties;
 import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.conf.DeprecatedPropertyUtil;
 import org.apache.accumulo.core.data.InstanceId;
@@ -65,6 +69,7 @@ import org.slf4j.LoggerFactory;
  * Provides a class for administering the accumulo instance
  */
 public class InstanceOperationsImpl implements InstanceOperations {
+
   private final ClientContext context;
 
   public InstanceOperationsImpl(ClientContext context) {
@@ -88,6 +93,33 @@ public class InstanceOperationsImpl implements InstanceOperations {
     checkLocalityGroups(property);
   }
 
+  @Override
+  public void modifyProperties(final Consumer<Map<String,String>> mapMutator)
+      throws AccumuloException, AccumuloSecurityException, IllegalArgumentException,
+      ConcurrentModificationException {
+    checkArgument(mapMutator != null, "mapMutator is null");
+
+    final TVersionedProperties vProperties = getSystemProperties();
+    mapMutator.accept(vProperties.getProperties());
+
+    for (Map.Entry<String,String> entry : vProperties.getProperties().entrySet()) {
+      final String property = Objects.requireNonNull(entry.getKey(), "property key is null");
+
+      DeprecatedPropertyUtil.getReplacementName(property, (log, replacement) -> {
+        // force a warning on the client side, but send the name the user used to the
+        // server-side
+        // to trigger a warning in the server logs, and to handle it there
+        log.warn("{} was deprecated and will be removed in a future release;"
+            + " setting its replacement {} instead", property, replacement);
+      });
+      checkLocalityGroups(property);
+    }
+
+    // Send to server
+    ThriftClientTypes.MANAGER.executeVoid(context, client -> client
+        .modifySystemProperties(TraceUtil.traceInfo(), context.rpcCreds(), vProperties));
+  }
+
   @Override
   public void removeProperty(final String property)
       throws AccumuloException, AccumuloSecurityException {
@@ -125,6 +157,13 @@ public class InstanceOperationsImpl implements InstanceOperations {
         .getConfiguration(TraceUtil.traceInfo(), context.rpcCreds(), ConfigurationType.CURRENT));
   }
 
+  @Override
+  public TVersionedProperties getSystemProperties()
+      throws AccumuloException, AccumuloSecurityException {
+    return ThriftClientTypes.CLIENT.execute(context,
+        client -> client.getVersionedSystemProperties(TraceUtil.traceInfo(), context.rpcCreds()));
+  }
+
   @Override
   public Map<String,String> getSiteConfiguration()
       throws AccumuloException, AccumuloSecurityException {
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/NamespaceOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/NamespaceOperationsImpl.java
index cf9429c389..b3c7660544 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/NamespaceOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/NamespaceOperationsImpl.java
@@ -34,6 +34,7 @@ import java.util.Map;
 import java.util.SortedSet;
 import java.util.TreeMap;
 import java.util.TreeSet;
+import java.util.function.Consumer;
 import java.util.stream.Collectors;
 
 import org.apache.accumulo.core.client.AccumuloException;
@@ -46,6 +47,7 @@ import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.admin.TableOperations;
 import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties;
 import org.apache.accumulo.core.clientImpl.thrift.TableOperationExceptionType;
 import org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException;
 import org.apache.accumulo.core.data.NamespaceId;
@@ -195,6 +197,36 @@ public class NamespaceOperationsImpl extends NamespaceOperationsHelper {
     checkLocalityGroups(namespace, property);
   }
 
+  @Override
+  public void modifyProperties(final String namespace,
+      final Consumer<Map<String,String>> mapMutator)
+      throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException {
+    EXISTING_NAMESPACE_NAME.validate(namespace);
+    checkArgument(mapMutator != null, "mapMutator is null");
+
+    final TVersionedProperties vProperties =
+        ThriftClientTypes.CLIENT.execute(context, client -> client
+            .getVersionedNamespaceProperties(TraceUtil.traceInfo(), context.rpcCreds(), namespace));
+    mapMutator.accept(vProperties.getProperties());
+
+    try {
+      // Send to server
+      ThriftClientTypes.MANAGER.executeVoidTableCommand(context,
+          client -> client.modifyNamespaceProperties(TraceUtil.traceInfo(), context.rpcCreds(),
+              namespace, vProperties));
+
+      for (String property : vProperties.getProperties().keySet()) {
+        checkLocalityGroups(namespace, property);
+      }
+
+    } catch (TableNotFoundException e) {
+      if (e.getCause() instanceof NamespaceNotFoundException)
+        throw (NamespaceNotFoundException) e.getCause();
+      else
+        throw new AccumuloException(e);
+    }
+  }
+
   @Override
   public void removeProperty(final String namespace, final String property)
       throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException {
@@ -235,6 +267,29 @@ public class NamespaceOperationsImpl extends NamespaceOperationsHelper {
     }
   }
 
+  @Override
+  public Map<String,String> getNamespaceProperties(String namespace)
+      throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException {
+    EXISTING_NAMESPACE_NAME.validate(namespace);
+
+    try {
+      return ThriftClientTypes.CLIENT.execute(context, client -> client
+          .getNamespaceProperties(TraceUtil.traceInfo(), context.rpcCreds(), namespace));
+    } catch (AccumuloException e) {
+      Throwable t = e.getCause();
+      if (t instanceof ThriftTableOperationException) {
+        ThriftTableOperationException ttoe = (ThriftTableOperationException) t;
+        if (ttoe.getType() == TableOperationExceptionType.NAMESPACE_NOTFOUND) {
+          throw new NamespaceNotFoundException(ttoe);
+        }
+        throw e;
+      }
+      throw e;
+    } catch (Exception e) {
+      throw new AccumuloException(e);
+    }
+  }
+
   @Override
   public Map<String,String> namespaceIdMap() {
     return Namespaces.getNameToIdMap(context).entrySet().stream()
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java
index 0b5a7f1eae..7b1912e9fa 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java
@@ -41,6 +41,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.ConcurrentModificationException;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -58,6 +59,7 @@ import java.util.TreeSet;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
 import java.util.function.Predicate;
 import java.util.regex.Pattern;
 import java.util.stream.Collectors;
@@ -93,6 +95,7 @@ import org.apache.accumulo.core.clientImpl.TabletLocator.TabletLocation;
 import org.apache.accumulo.core.clientImpl.bulk.BulkImport;
 import org.apache.accumulo.core.clientImpl.thrift.ClientService.Client;
 import org.apache.accumulo.core.clientImpl.thrift.TDiskUsage;
+import org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties;
 import org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException;
 import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException;
@@ -1007,6 +1010,31 @@ public class TableOperationsImpl extends TableOperationsHelper {
     }
   }
 
+  @Override
+  public void modifyProperties(String tableName, final Consumer<Map<String,String>> mapMutator)
+      throws AccumuloException, AccumuloSecurityException, IllegalArgumentException,
+      ConcurrentModificationException {
+    EXISTING_TABLE_NAME.validate(tableName);
+    checkArgument(mapMutator != null, "mapMutator is null");
+
+    final TVersionedProperties vProperties =
+        ThriftClientTypes.CLIENT.execute(context, client -> client
+            .getVersionedTableProperties(TraceUtil.traceInfo(), context.rpcCreds(), tableName));
+    mapMutator.accept(vProperties.getProperties());
+
+    try {
+      // Send to server
+      ThriftClientTypes.MANAGER.executeVoid(context,
+          client -> client.modifyTableProperties(TraceUtil.traceInfo(), context.rpcCreds(),
+              tableName, vProperties));
+      for (String property : vProperties.getProperties().keySet()) {
+        checkLocalityGroups(tableName, property);
+      }
+    } catch (TableNotFoundException e) {
+      throw new AccumuloException(e);
+    }
+  }
+
   private void setPropertyNoChecks(final String tableName, final String property,
       final String value)
       throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
@@ -1079,6 +1107,33 @@ public class TableOperationsImpl extends TableOperationsHelper {
     }
   }
 
+  @Override
+  public Map<String,String> getTableProperties(final String tableName)
+      throws AccumuloException, TableNotFoundException {
+    EXISTING_TABLE_NAME.validate(tableName);
+
+    try {
+      return ThriftClientTypes.CLIENT.execute(context, client -> client
+          .getTableProperties(TraceUtil.traceInfo(), context.rpcCreds(), tableName));
+    } catch (AccumuloException e) {
+      Throwable t = e.getCause();
+      if (t instanceof ThriftTableOperationException) {
+        ThriftTableOperationException ttoe = (ThriftTableOperationException) t;
+        switch (ttoe.getType()) {
+          case NOTFOUND:
+            throw new TableNotFoundException(ttoe);
+          case NAMESPACE_NOTFOUND:
+            throw new TableNotFoundException(tableName, new NamespaceNotFoundException(ttoe));
+          default:
+            throw e;
+        }
+      }
+      throw e;
+    } catch (Exception e) {
+      throw new AccumuloException(e);
+    }
+  }
+
   @Override
   public void setLocalityGroups(String tableName, Map<String,Set<Text>> groups)
       throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/clients/ManagerThriftClient.java b/core/src/main/java/org/apache/accumulo/core/rpc/clients/ManagerThriftClient.java
index cc942568b6..2e92b21a6c 100644
--- a/core/src/main/java/org/apache/accumulo/core/rpc/clients/ManagerThriftClient.java
+++ b/core/src/main/java/org/apache/accumulo/core/rpc/clients/ManagerThriftClient.java
@@ -21,11 +21,14 @@ package org.apache.accumulo.core.rpc.clients;
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
 
+import java.util.ConcurrentModificationException;
+
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.NamespaceNotFoundException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.clientImpl.ClientContext;
+import org.apache.accumulo.core.clientImpl.thrift.ThriftConcurrentModificationException;
 import org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException;
 import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException;
@@ -74,6 +77,8 @@ public class ManagerThriftClient extends ThriftClientTypes<Client>
         // Let it loop, fetching a new location
         LOG.debug("Contacted a Manager which is no longer active, retrying");
         sleepUninterruptibly(100, MILLISECONDS);
+      } catch (ThriftConcurrentModificationException e) {
+        throw new ConcurrentModificationException(e.getMessage(), e);
       } catch (Exception e) {
         throw new AccumuloException(e);
       } finally {
@@ -119,6 +124,8 @@ public class ManagerThriftClient extends ThriftClientTypes<Client>
         // Let it loop, fetching a new location
         LOG.debug("Contacted a Manager which is no longer active, retrying");
         sleepUninterruptibly(100, MILLISECONDS);
+      } catch (ThriftConcurrentModificationException e) {
+        throw new ConcurrentModificationException(e.getMessage(), e);
       } catch (Exception e) {
         throw new AccumuloException(e);
       } finally {
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/clientImpl/thrift/ClientService.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/clientImpl/thrift/ClientService.java
index c1d2c505fb..8a511d3d66 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/clientImpl/thrift/ClientService.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/clientImpl/thrift/ClientService.java
@@ -79,10 +79,22 @@ public class ClientService {
 
     public java.util.Map<java.lang.String,java.lang.String> getConfiguration(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, ConfigurationType type) throws org.apache.thrift.TException;
 
+    public java.util.Map<java.lang.String,java.lang.String> getSystemProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException;
+
+    public TVersionedProperties getVersionedSystemProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException;
+
     public java.util.Map<java.lang.String,java.lang.String> getTableConfiguration(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName) throws ThriftTableOperationException, org.apache.thrift.TException;
 
+    public java.util.Map<java.lang.String,java.lang.String> getTableProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName) throws ThriftTableOperationException, org.apache.thrift.TException;
+
+    public TVersionedProperties getVersionedTableProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName) throws ThriftTableOperationException, org.apache.thrift.TException;
+
     public java.util.Map<java.lang.String,java.lang.String> getNamespaceConfiguration(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns) throws ThriftTableOperationException, org.apache.thrift.TException;
 
+    public java.util.Map<java.lang.String,java.lang.String> getNamespaceProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns) throws ThriftTableOperationException, org.apache.thrift.TException;
+
+    public TVersionedProperties getVersionedNamespaceProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns) throws ThriftTableOperationException, org.apache.thrift.TException;
+
     public boolean checkClass(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String className, java.lang.String interfaceMatch) throws org.apache.thrift.TException;
 
     public boolean checkTableClass(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableId, java.lang.String className, java.lang.String interfaceMatch) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException;
@@ -143,10 +155,22 @@ public class ClientService {
 
     public void getConfiguration(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, ConfigurationType type, org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>> resultHandler) throws org.apache.thrift.TException;
 
+    public void getSystemProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>> resultHandler) throws org.apache.thrift.TException;
+
+    public void getVersionedSystemProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<TVersionedProperties> resultHandler) throws org.apache.thrift.TException;
+
     public void getTableConfiguration(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>> resultHandler) throws org.apache.thrift.TException;
 
+    public void getTableProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>> resultHandler) throws org.apache.thrift.TException;
+
+    public void getVersionedTableProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.thrift.async.AsyncMethodCallback<TVersionedProperties> resultHandler) throws org.apache.thrift.TException;
+
     public void getNamespaceConfiguration(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>> resultHandler) throws org.apache.thrift.TException;
 
+    public void getNamespaceProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>> resultHandler) throws org.apache.thrift.TException;
+
+    public void getVersionedNamespaceProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, org.apache.thrift.async.AsyncMethodCallback<TVersionedProperties> resultHandler) throws org.apache.thrift.TException;
+
     public void checkClass(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String className, java.lang.String interfaceMatch, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException;
 
     public void checkTableClass(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableId, java.lang.String className, java.lang.String interfaceMatch, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException;
@@ -858,6 +882,54 @@ public class ClientService {
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getConfiguration failed: unknown result");
     }
 
+    public java.util.Map<java.lang.String,java.lang.String> getSystemProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
+    {
+      send_getSystemProperties(tinfo, credentials);
+      return recv_getSystemProperties();
+    }
+
+    public void send_getSystemProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
+    {
+      getSystemProperties_args args = new getSystemProperties_args();
+      args.setTinfo(tinfo);
+      args.setCredentials(credentials);
+      sendBase("getSystemProperties", args);
+    }
+
+    public java.util.Map<java.lang.String,java.lang.String> recv_getSystemProperties() throws org.apache.thrift.TException
+    {
+      getSystemProperties_result result = new getSystemProperties_result();
+      receiveBase(result, "getSystemProperties");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getSystemProperties failed: unknown result");
+    }
+
+    public TVersionedProperties getVersionedSystemProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
+    {
+      send_getVersionedSystemProperties(tinfo, credentials);
+      return recv_getVersionedSystemProperties();
+    }
+
+    public void send_getVersionedSystemProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
+    {
+      getVersionedSystemProperties_args args = new getVersionedSystemProperties_args();
+      args.setTinfo(tinfo);
+      args.setCredentials(credentials);
+      sendBase("getVersionedSystemProperties", args);
+    }
+
+    public TVersionedProperties recv_getVersionedSystemProperties() throws org.apache.thrift.TException
+    {
+      getVersionedSystemProperties_result result = new getVersionedSystemProperties_result();
+      receiveBase(result, "getVersionedSystemProperties");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getVersionedSystemProperties failed: unknown result");
+    }
+
     public java.util.Map<java.lang.String,java.lang.String> getTableConfiguration(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName) throws ThriftTableOperationException, org.apache.thrift.TException
     {
       send_getTableConfiguration(tinfo, credentials, tableName);
@@ -886,6 +958,62 @@ public class ClientService {
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getTableConfiguration failed: unknown result");
     }
 
+    public java.util.Map<java.lang.String,java.lang.String> getTableProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName) throws ThriftTableOperationException, org.apache.thrift.TException
+    {
+      send_getTableProperties(tinfo, credentials, tableName);
+      return recv_getTableProperties();
+    }
+
+    public void send_getTableProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName) throws org.apache.thrift.TException
+    {
+      getTableProperties_args args = new getTableProperties_args();
+      args.setTinfo(tinfo);
+      args.setCredentials(credentials);
+      args.setTableName(tableName);
+      sendBase("getTableProperties", args);
+    }
+
+    public java.util.Map<java.lang.String,java.lang.String> recv_getTableProperties() throws ThriftTableOperationException, org.apache.thrift.TException
+    {
+      getTableProperties_result result = new getTableProperties_result();
+      receiveBase(result, "getTableProperties");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.tope != null) {
+        throw result.tope;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getTableProperties failed: unknown result");
+    }
+
+    public TVersionedProperties getVersionedTableProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName) throws ThriftTableOperationException, org.apache.thrift.TException
+    {
+      send_getVersionedTableProperties(tinfo, credentials, tableName);
+      return recv_getVersionedTableProperties();
+    }
+
+    public void send_getVersionedTableProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName) throws org.apache.thrift.TException
+    {
+      getVersionedTableProperties_args args = new getVersionedTableProperties_args();
+      args.setTinfo(tinfo);
+      args.setCredentials(credentials);
+      args.setTableName(tableName);
+      sendBase("getVersionedTableProperties", args);
+    }
+
+    public TVersionedProperties recv_getVersionedTableProperties() throws ThriftTableOperationException, org.apache.thrift.TException
+    {
+      getVersionedTableProperties_result result = new getVersionedTableProperties_result();
+      receiveBase(result, "getVersionedTableProperties");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.tope != null) {
+        throw result.tope;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getVersionedTableProperties failed: unknown result");
+    }
+
     public java.util.Map<java.lang.String,java.lang.String> getNamespaceConfiguration(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns) throws ThriftTableOperationException, org.apache.thrift.TException
     {
       send_getNamespaceConfiguration(tinfo, credentials, ns);
@@ -914,6 +1042,62 @@ public class ClientService {
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getNamespaceConfiguration failed: unknown result");
     }
 
+    public java.util.Map<java.lang.String,java.lang.String> getNamespaceProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns) throws ThriftTableOperationException, org.apache.thrift.TException
+    {
+      send_getNamespaceProperties(tinfo, credentials, ns);
+      return recv_getNamespaceProperties();
+    }
+
+    public void send_getNamespaceProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns) throws org.apache.thrift.TException
+    {
+      getNamespaceProperties_args args = new getNamespaceProperties_args();
+      args.setTinfo(tinfo);
+      args.setCredentials(credentials);
+      args.setNs(ns);
+      sendBase("getNamespaceProperties", args);
+    }
+
+    public java.util.Map<java.lang.String,java.lang.String> recv_getNamespaceProperties() throws ThriftTableOperationException, org.apache.thrift.TException
+    {
+      getNamespaceProperties_result result = new getNamespaceProperties_result();
+      receiveBase(result, "getNamespaceProperties");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.tope != null) {
+        throw result.tope;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getNamespaceProperties failed: unknown result");
+    }
+
+    public TVersionedProperties getVersionedNamespaceProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns) throws ThriftTableOperationException, org.apache.thrift.TException
+    {
+      send_getVersionedNamespaceProperties(tinfo, credentials, ns);
+      return recv_getVersionedNamespaceProperties();
+    }
+
+    public void send_getVersionedNamespaceProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns) throws org.apache.thrift.TException
+    {
+      getVersionedNamespaceProperties_args args = new getVersionedNamespaceProperties_args();
+      args.setTinfo(tinfo);
+      args.setCredentials(credentials);
+      args.setNs(ns);
+      sendBase("getVersionedNamespaceProperties", args);
+    }
+
+    public TVersionedProperties recv_getVersionedNamespaceProperties() throws ThriftTableOperationException, org.apache.thrift.TException
+    {
+      getVersionedNamespaceProperties_result result = new getVersionedNamespaceProperties_result();
+      receiveBase(result, "getVersionedNamespaceProperties");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.tope != null) {
+        throw result.tope;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getVersionedNamespaceProperties failed: unknown result");
+    }
+
     public boolean checkClass(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String className, java.lang.String interfaceMatch) throws org.apache.thrift.TException
     {
       send_checkClass(tinfo, credentials, className, interfaceMatch);
@@ -1995,6 +2179,76 @@ public class ClientService {
       }
     }
 
+    public void getSystemProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      getSystemProperties_call method_call = new getSystemProperties_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class getSystemProperties_call extends org.apache.thrift.async.TAsyncMethodCall<java.util.Map<java.lang.String,java.lang.String>> {
+      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
+      public getSystemProperties_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,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);
+        this.tinfo = tinfo;
+        this.credentials = credentials;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getSystemProperties", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        getSystemProperties_args args = new getSystemProperties_args();
+        args.setTinfo(tinfo);
+        args.setCredentials(credentials);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public java.util.Map<java.lang.String,java.lang.String> getResult() throws org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new java.lang.IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_getSystemProperties();
+      }
+    }
+
+    public void getVersionedSystemProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<TVersionedProperties> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      getVersionedSystemProperties_call method_call = new getVersionedSystemProperties_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class getVersionedSystemProperties_call extends org.apache.thrift.async.TAsyncMethodCall<TVersionedProperties> {
+      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
+      public getVersionedSystemProperties_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<TVersionedProperties> 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);
+        this.tinfo = tinfo;
+        this.credentials = credentials;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getVersionedSystemProperties", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        getVersionedSystemProperties_args args = new getVersionedSystemProperties_args();
+        args.setTinfo(tinfo);
+        args.setCredentials(credentials);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public TVersionedProperties getResult() throws org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new java.lang.IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_getVersionedSystemProperties();
+      }
+    }
+
     public void getTableConfiguration(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       getTableConfiguration_call method_call = new getTableConfiguration_call(tinfo, credentials, tableName, resultHandler, this, ___protocolFactory, ___transport);
@@ -2033,6 +2287,82 @@ public class ClientService {
       }
     }
 
+    public void getTableProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      getTableProperties_call method_call = new getTableProperties_call(tinfo, credentials, tableName, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class getTableProperties_call extends org.apache.thrift.async.TAsyncMethodCall<java.util.Map<java.lang.String,java.lang.String>> {
+      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
+      private java.lang.String tableName;
+      public getTableProperties_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,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.TExc [...]
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.tinfo = tinfo;
+        this.credentials = credentials;
+        this.tableName = tableName;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getTableProperties", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        getTableProperties_args args = new getTableProperties_args();
+        args.setTinfo(tinfo);
+        args.setCredentials(credentials);
+        args.setTableName(tableName);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public java.util.Map<java.lang.String,java.lang.String> getResult() throws ThriftTableOperationException, 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_getTableProperties();
+      }
+    }
+
+    public void getVersionedTableProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.thrift.async.AsyncMethodCallback<TVersionedProperties> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      getVersionedTableProperties_call method_call = new getVersionedTableProperties_call(tinfo, credentials, tableName, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class getVersionedTableProperties_call extends org.apache.thrift.async.TAsyncMethodCall<TVersionedProperties> {
+      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
+      private java.lang.String tableName;
+      public getVersionedTableProperties_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.thrift.async.AsyncMethodCallback<TVersionedProperties> 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);
+        this.tinfo = tinfo;
+        this.credentials = credentials;
+        this.tableName = tableName;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getVersionedTableProperties", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        getVersionedTableProperties_args args = new getVersionedTableProperties_args();
+        args.setTinfo(tinfo);
+        args.setCredentials(credentials);
+        args.setTableName(tableName);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public TVersionedProperties getResult() throws ThriftTableOperationException, 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_getVersionedTableProperties();
+      }
+    }
+
     public void getNamespaceConfiguration(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       getNamespaceConfiguration_call method_call = new getNamespaceConfiguration_call(tinfo, credentials, ns, resultHandler, this, ___protocolFactory, ___transport);
@@ -2071,6 +2401,82 @@ public class ClientService {
       }
     }
 
+    public void getNamespaceProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      getNamespaceProperties_call method_call = new getNamespaceProperties_call(tinfo, credentials, ns, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class getNamespaceProperties_call extends org.apache.thrift.async.TAsyncMethodCall<java.util.Map<java.lang.String,java.lang.String>> {
+      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
+      private java.lang.String ns;
+      public getNamespaceProperties_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,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);
+        this.tinfo = tinfo;
+        this.credentials = credentials;
+        this.ns = ns;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getNamespaceProperties", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        getNamespaceProperties_args args = new getNamespaceProperties_args();
+        args.setTinfo(tinfo);
+        args.setCredentials(credentials);
+        args.setNs(ns);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public java.util.Map<java.lang.String,java.lang.String> getResult() throws ThriftTableOperationException, 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_getNamespaceProperties();
+      }
+    }
+
+    public void getVersionedNamespaceProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, org.apache.thrift.async.AsyncMethodCallback<TVersionedProperties> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      getVersionedNamespaceProperties_call method_call = new getVersionedNamespaceProperties_call(tinfo, credentials, ns, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class getVersionedNamespaceProperties_call extends org.apache.thrift.async.TAsyncMethodCall<TVersionedProperties> {
+      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
+      private java.lang.String ns;
+      public getVersionedNamespaceProperties_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, org.apache.thrift.async.AsyncMethodCallback<TVersionedProperties> 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);
+        this.tinfo = tinfo;
+        this.credentials = credentials;
+        this.ns = ns;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getVersionedNamespaceProperties", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        getVersionedNamespaceProperties_args args = new getVersionedNamespaceProperties_args();
+        args.setTinfo(tinfo);
+        args.setCredentials(credentials);
+        args.setNs(ns);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public TVersionedProperties getResult() throws ThriftTableOperationException, 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_getVersionedNamespaceProperties();
+      }
+    }
+
     public void checkClass(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String className, java.lang.String interfaceMatch, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       checkClass_call method_call = new checkClass_call(tinfo, credentials, className, interfaceMatch, resultHandler, this, ___protocolFactory, ___transport);
@@ -2238,8 +2644,14 @@ public class ClientService {
       processMap.put("grantNamespacePermission", new grantNamespacePermission());
       processMap.put("revokeNamespacePermission", new revokeNamespacePermission());
       processMap.put("getConfiguration", new getConfiguration());
+      processMap.put("getSystemProperties", new getSystemProperties());
+      processMap.put("getVersionedSystemProperties", new getVersionedSystemProperties());
       processMap.put("getTableConfiguration", new getTableConfiguration());
+      processMap.put("getTableProperties", new getTableProperties());
+      processMap.put("getVersionedTableProperties", new getVersionedTableProperties());
       processMap.put("getNamespaceConfiguration", new getNamespaceConfiguration());
+      processMap.put("getNamespaceProperties", new getNamespaceProperties());
+      processMap.put("getVersionedNamespaceProperties", new getVersionedNamespaceProperties());
       processMap.put("checkClass", new checkClass());
       processMap.put("checkTableClass", new checkTableClass());
       processMap.put("checkNamespaceClass", new checkNamespaceClass());
@@ -2973,6 +3385,56 @@ public class ClientService {
       }
     }
 
+    public static class getSystemProperties<I extends Iface> extends org.apache.thrift.ProcessFunction<I, getSystemProperties_args> {
+      public getSystemProperties() {
+        super("getSystemProperties");
+      }
+
+      public getSystemProperties_args getEmptyArgsInstance() {
+        return new getSystemProperties_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      @Override
+      protected boolean rethrowUnhandledExceptions() {
+        return false;
+      }
+
+      public getSystemProperties_result getResult(I iface, getSystemProperties_args args) throws org.apache.thrift.TException {
+        getSystemProperties_result result = new getSystemProperties_result();
+        result.success = iface.getSystemProperties(args.tinfo, args.credentials);
+        return result;
+      }
+    }
+
+    public static class getVersionedSystemProperties<I extends Iface> extends org.apache.thrift.ProcessFunction<I, getVersionedSystemProperties_args> {
+      public getVersionedSystemProperties() {
+        super("getVersionedSystemProperties");
+      }
+
+      public getVersionedSystemProperties_args getEmptyArgsInstance() {
+        return new getVersionedSystemProperties_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      @Override
+      protected boolean rethrowUnhandledExceptions() {
+        return false;
+      }
+
+      public getVersionedSystemProperties_result getResult(I iface, getVersionedSystemProperties_args args) throws org.apache.thrift.TException {
+        getVersionedSystemProperties_result result = new getVersionedSystemProperties_result();
+        result.success = iface.getVersionedSystemProperties(args.tinfo, args.credentials);
+        return result;
+      }
+    }
+
     public static class getTableConfiguration<I extends Iface> extends org.apache.thrift.ProcessFunction<I, getTableConfiguration_args> {
       public getTableConfiguration() {
         super("getTableConfiguration");
@@ -3002,6 +3464,64 @@ public class ClientService {
       }
     }
 
+    public static class getTableProperties<I extends Iface> extends org.apache.thrift.ProcessFunction<I, getTableProperties_args> {
+      public getTableProperties() {
+        super("getTableProperties");
+      }
+
+      public getTableProperties_args getEmptyArgsInstance() {
+        return new getTableProperties_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      @Override
+      protected boolean rethrowUnhandledExceptions() {
+        return false;
+      }
+
+      public getTableProperties_result getResult(I iface, getTableProperties_args args) throws org.apache.thrift.TException {
+        getTableProperties_result result = new getTableProperties_result();
+        try {
+          result.success = iface.getTableProperties(args.tinfo, args.credentials, args.tableName);
+        } catch (ThriftTableOperationException tope) {
+          result.tope = tope;
+        }
+        return result;
+      }
+    }
+
+    public static class getVersionedTableProperties<I extends Iface> extends org.apache.thrift.ProcessFunction<I, getVersionedTableProperties_args> {
+      public getVersionedTableProperties() {
+        super("getVersionedTableProperties");
+      }
+
+      public getVersionedTableProperties_args getEmptyArgsInstance() {
+        return new getVersionedTableProperties_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      @Override
+      protected boolean rethrowUnhandledExceptions() {
+        return false;
+      }
+
+      public getVersionedTableProperties_result getResult(I iface, getVersionedTableProperties_args args) throws org.apache.thrift.TException {
+        getVersionedTableProperties_result result = new getVersionedTableProperties_result();
+        try {
+          result.success = iface.getVersionedTableProperties(args.tinfo, args.credentials, args.tableName);
+        } catch (ThriftTableOperationException tope) {
+          result.tope = tope;
+        }
+        return result;
+      }
+    }
+
     public static class getNamespaceConfiguration<I extends Iface> extends org.apache.thrift.ProcessFunction<I, getNamespaceConfiguration_args> {
       public getNamespaceConfiguration() {
         super("getNamespaceConfiguration");
@@ -3031,6 +3551,64 @@ public class ClientService {
       }
     }
 
+    public static class getNamespaceProperties<I extends Iface> extends org.apache.thrift.ProcessFunction<I, getNamespaceProperties_args> {
+      public getNamespaceProperties() {
+        super("getNamespaceProperties");
+      }
+
+      public getNamespaceProperties_args getEmptyArgsInstance() {
+        return new getNamespaceProperties_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      @Override
+      protected boolean rethrowUnhandledExceptions() {
+        return false;
+      }
+
+      public getNamespaceProperties_result getResult(I iface, getNamespaceProperties_args args) throws org.apache.thrift.TException {
+        getNamespaceProperties_result result = new getNamespaceProperties_result();
+        try {
+          result.success = iface.getNamespaceProperties(args.tinfo, args.credentials, args.ns);
+        } catch (ThriftTableOperationException tope) {
+          result.tope = tope;
+        }
+        return result;
+      }
+    }
+
+    public static class getVersionedNamespaceProperties<I extends Iface> extends org.apache.thrift.ProcessFunction<I, getVersionedNamespaceProperties_args> {
+      public getVersionedNamespaceProperties() {
+        super("getVersionedNamespaceProperties");
+      }
+
+      public getVersionedNamespaceProperties_args getEmptyArgsInstance() {
+        return new getVersionedNamespaceProperties_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      @Override
+      protected boolean rethrowUnhandledExceptions() {
+        return false;
+      }
+
+      public getVersionedNamespaceProperties_result getResult(I iface, getVersionedNamespaceProperties_args args) throws org.apache.thrift.TException {
+        getVersionedNamespaceProperties_result result = new getVersionedNamespaceProperties_result();
+        try {
+          result.success = iface.getVersionedNamespaceProperties(args.tinfo, args.credentials, args.ns);
+        } catch (ThriftTableOperationException tope) {
+          result.tope = tope;
+        }
+        return result;
+      }
+    }
+
     public static class checkClass<I extends Iface> extends org.apache.thrift.ProcessFunction<I, checkClass_args> {
       public checkClass() {
         super("checkClass");
@@ -3159,8 +3737,14 @@ public class ClientService {
       processMap.put("grantNamespacePermission", new grantNamespacePermission());
       processMap.put("revokeNamespacePermission", new revokeNamespacePermission());
       processMap.put("getConfiguration", new getConfiguration());
+      processMap.put("getSystemProperties", new getSystemProperties());
+      processMap.put("getVersionedSystemProperties", new getVersionedSystemProperties());
       processMap.put("getTableConfiguration", new getTableConfiguration());
+      processMap.put("getTableProperties", new getTableProperties());
+      processMap.put("getVersionedTableProperties", new getVersionedTableProperties());
       processMap.put("getNamespaceConfiguration", new getNamespaceConfiguration());
+      processMap.put("getNamespaceProperties", new getNamespaceProperties());
+      processMap.put("getVersionedNamespaceProperties", new getVersionedNamespaceProperties());
       processMap.put("checkClass", new checkClass());
       processMap.put("checkTableClass", new checkTableClass());
       processMap.put("checkNamespaceClass", new checkNamespaceClass());
@@ -4799,85 +5383,20 @@ public class ClientService {
       }
     }
 
-    public static class getTableConfiguration<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getTableConfiguration_args, java.util.Map<java.lang.String,java.lang.String>> {
-      public getTableConfiguration() {
-        super("getTableConfiguration");
-      }
-
-      public getTableConfiguration_args getEmptyArgsInstance() {
-        return new getTableConfiguration_args();
-      }
-
-      public org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,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.Map<java.lang.String,java.lang.String>>() { 
-          public void onComplete(java.util.Map<java.lang.String,java.lang.String> o) {
-            getTableConfiguration_result result = new getTableConfiguration_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;
-            getTableConfiguration_result result = new getTableConfiguration_result();
-            if (e instanceof ThriftTableOperationException) {
-              result.tope = (ThriftTableOperationException) e;
-              result.setTopeIsSet(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, getTableConfiguration_args args, org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>> resultHandler) throws org.apache.thrift.TException {
-        iface.getTableConfiguration(args.tinfo, args.credentials, args.tableName,resultHandler);
-      }
-    }
-
-    public static class getNamespaceConfiguration<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getNamespaceConfiguration_args, java.util.Map<java.lang.String,java.lang.String>> {
-      public getNamespaceConfiguration() {
-        super("getNamespaceConfiguration");
+    public static class getSystemProperties<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getSystemProperties_args, java.util.Map<java.lang.String,java.lang.String>> {
+      public getSystemProperties() {
+        super("getSystemProperties");
       }
 
-      public getNamespaceConfiguration_args getEmptyArgsInstance() {
-        return new getNamespaceConfiguration_args();
+      public getSystemProperties_args getEmptyArgsInstance() {
+        return new getSystemProperties_args();
       }
 
       public org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,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.Map<java.lang.String,java.lang.String>>() { 
           public void onComplete(java.util.Map<java.lang.String,java.lang.String> o) {
-            getNamespaceConfiguration_result result = new getNamespaceConfiguration_result();
+            getSystemProperties_result result = new getSystemProperties_result();
             result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -4892,12 +5411,8 @@ public class ClientService {
           public void onError(java.lang.Exception e) {
             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
             org.apache.thrift.TSerializable msg;
-            getNamespaceConfiguration_result result = new getNamespaceConfiguration_result();
-            if (e instanceof ThriftTableOperationException) {
-              result.tope = (ThriftTableOperationException) e;
-              result.setTopeIsSet(true);
-              msg = result;
-            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
+            getSystemProperties_result result = new getSystemProperties_result();
+            if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
               return;
@@ -4924,27 +5439,26 @@ public class ClientService {
         return false;
       }
 
-      public void start(I iface, getNamespaceConfiguration_args args, org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>> resultHandler) throws org.apache.thrift.TException {
-        iface.getNamespaceConfiguration(args.tinfo, args.credentials, args.ns,resultHandler);
+      public void start(I iface, getSystemProperties_args args, org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>> resultHandler) throws org.apache.thrift.TException {
+        iface.getSystemProperties(args.tinfo, args.credentials,resultHandler);
       }
     }
 
-    public static class checkClass<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, checkClass_args, java.lang.Boolean> {
-      public checkClass() {
-        super("checkClass");
+    public static class getVersionedSystemProperties<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getVersionedSystemProperties_args, TVersionedProperties> {
+      public getVersionedSystemProperties() {
+        super("getVersionedSystemProperties");
       }
 
-      public checkClass_args getEmptyArgsInstance() {
-        return new checkClass_args();
+      public getVersionedSystemProperties_args getEmptyArgsInstance() {
+        return new getVersionedSystemProperties_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<TVersionedProperties> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
         final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean>() { 
-          public void onComplete(java.lang.Boolean o) {
-            checkClass_result result = new checkClass_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<TVersionedProperties>() { 
+          public void onComplete(TVersionedProperties o) {
+            getVersionedSystemProperties_result result = new getVersionedSystemProperties_result();
             result.success = o;
-            result.setSuccessIsSet(true);
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -4958,7 +5472,7 @@ public class ClientService {
           public void onError(java.lang.Exception e) {
             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
             org.apache.thrift.TSerializable msg;
-            checkClass_result result = new checkClass_result();
+            getVersionedSystemProperties_result result = new getVersionedSystemProperties_result();
             if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
@@ -4986,27 +5500,26 @@ public class ClientService {
         return false;
       }
 
-      public void start(I iface, checkClass_args args, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
-        iface.checkClass(args.tinfo, args.credentials, args.className, args.interfaceMatch,resultHandler);
+      public void start(I iface, getVersionedSystemProperties_args args, org.apache.thrift.async.AsyncMethodCallback<TVersionedProperties> resultHandler) throws org.apache.thrift.TException {
+        iface.getVersionedSystemProperties(args.tinfo, args.credentials,resultHandler);
       }
     }
 
-    public static class checkTableClass<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, checkTableClass_args, java.lang.Boolean> {
-      public checkTableClass() {
-        super("checkTableClass");
+    public static class getTableConfiguration<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getTableConfiguration_args, java.util.Map<java.lang.String,java.lang.String>> {
+      public getTableConfiguration() {
+        super("getTableConfiguration");
       }
 
-      public checkTableClass_args getEmptyArgsInstance() {
-        return new checkTableClass_args();
+      public getTableConfiguration_args getEmptyArgsInstance() {
+        return new getTableConfiguration_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
         final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean>() { 
-          public void onComplete(java.lang.Boolean o) {
-            checkTableClass_result result = new checkTableClass_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>>() { 
+          public void onComplete(java.util.Map<java.lang.String,java.lang.String> o) {
+            getTableConfiguration_result result = new getTableConfiguration_result();
             result.success = o;
-            result.setSuccessIsSet(true);
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -5020,12 +5533,8 @@ public class ClientService {
           public void onError(java.lang.Exception e) {
             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
             org.apache.thrift.TSerializable msg;
-            checkTableClass_result result = new checkTableClass_result();
-            if (e instanceof ThriftSecurityException) {
-              result.sec = (ThriftSecurityException) e;
-              result.setSecIsSet(true);
-              msg = result;
-            } else if (e instanceof ThriftTableOperationException) {
+            getTableConfiguration_result result = new getTableConfiguration_result();
+            if (e instanceof ThriftTableOperationException) {
               result.tope = (ThriftTableOperationException) e;
               result.setTopeIsSet(true);
               msg = result;
@@ -5056,27 +5565,26 @@ public class ClientService {
         return false;
       }
 
-      public void start(I iface, checkTableClass_args args, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
-        iface.checkTableClass(args.tinfo, args.credentials, args.tableId, args.className, args.interfaceMatch,resultHandler);
+      public void start(I iface, getTableConfiguration_args args, org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>> resultHandler) throws org.apache.thrift.TException {
+        iface.getTableConfiguration(args.tinfo, args.credentials, args.tableName,resultHandler);
       }
     }
 
-    public static class checkNamespaceClass<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, checkNamespaceClass_args, java.lang.Boolean> {
-      public checkNamespaceClass() {
-        super("checkNamespaceClass");
+    public static class getTableProperties<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getTableProperties_args, java.util.Map<java.lang.String,java.lang.String>> {
+      public getTableProperties() {
+        super("getTableProperties");
       }
 
-      public checkNamespaceClass_args getEmptyArgsInstance() {
-        return new checkNamespaceClass_args();
+      public getTableProperties_args getEmptyArgsInstance() {
+        return new getTableProperties_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
         final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean>() { 
-          public void onComplete(java.lang.Boolean o) {
-            checkNamespaceClass_result result = new checkNamespaceClass_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>>() { 
+          public void onComplete(java.util.Map<java.lang.String,java.lang.String> o) {
+            getTableProperties_result result = new getTableProperties_result();
             result.success = o;
-            result.setSuccessIsSet(true);
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -5090,12 +5598,470 @@ public class ClientService {
           public void onError(java.lang.Exception e) {
             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
             org.apache.thrift.TSerializable msg;
-            checkNamespaceClass_result result = new checkNamespaceClass_result();
-            if (e instanceof ThriftSecurityException) {
-              result.sec = (ThriftSecurityException) e;
-              result.setSecIsSet(true);
-              msg = result;
-            } else if (e instanceof ThriftTableOperationException) {
+            getTableProperties_result result = new getTableProperties_result();
+            if (e instanceof ThriftTableOperationException) {
+              result.tope = (ThriftTableOperationException) e;
+              result.setTopeIsSet(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, getTableProperties_args args, org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>> resultHandler) throws org.apache.thrift.TException {
+        iface.getTableProperties(args.tinfo, args.credentials, args.tableName,resultHandler);
+      }
+    }
+
+    public static class getVersionedTableProperties<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getVersionedTableProperties_args, TVersionedProperties> {
+      public getVersionedTableProperties() {
+        super("getVersionedTableProperties");
+      }
+
+      public getVersionedTableProperties_args getEmptyArgsInstance() {
+        return new getVersionedTableProperties_args();
+      }
+
+      public org.apache.thrift.async.AsyncMethodCallback<TVersionedProperties> 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<TVersionedProperties>() { 
+          public void onComplete(TVersionedProperties o) {
+            getVersionedTableProperties_result result = new getVersionedTableProperties_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;
+            getVersionedTableProperties_result result = new getVersionedTableProperties_result();
+            if (e instanceof ThriftTableOperationException) {
+              result.tope = (ThriftTableOperationException) e;
+              result.setTopeIsSet(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, getVersionedTableProperties_args args, org.apache.thrift.async.AsyncMethodCallback<TVersionedProperties> resultHandler) throws org.apache.thrift.TException {
+        iface.getVersionedTableProperties(args.tinfo, args.credentials, args.tableName,resultHandler);
+      }
+    }
+
+    public static class getNamespaceConfiguration<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getNamespaceConfiguration_args, java.util.Map<java.lang.String,java.lang.String>> {
+      public getNamespaceConfiguration() {
+        super("getNamespaceConfiguration");
+      }
+
+      public getNamespaceConfiguration_args getEmptyArgsInstance() {
+        return new getNamespaceConfiguration_args();
+      }
+
+      public org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,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.Map<java.lang.String,java.lang.String>>() { 
+          public void onComplete(java.util.Map<java.lang.String,java.lang.String> o) {
+            getNamespaceConfiguration_result result = new getNamespaceConfiguration_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;
+            getNamespaceConfiguration_result result = new getNamespaceConfiguration_result();
+            if (e instanceof ThriftTableOperationException) {
+              result.tope = (ThriftTableOperationException) e;
+              result.setTopeIsSet(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, getNamespaceConfiguration_args args, org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>> resultHandler) throws org.apache.thrift.TException {
+        iface.getNamespaceConfiguration(args.tinfo, args.credentials, args.ns,resultHandler);
+      }
+    }
+
+    public static class getNamespaceProperties<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getNamespaceProperties_args, java.util.Map<java.lang.String,java.lang.String>> {
+      public getNamespaceProperties() {
+        super("getNamespaceProperties");
+      }
+
+      public getNamespaceProperties_args getEmptyArgsInstance() {
+        return new getNamespaceProperties_args();
+      }
+
+      public org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,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.Map<java.lang.String,java.lang.String>>() { 
+          public void onComplete(java.util.Map<java.lang.String,java.lang.String> o) {
+            getNamespaceProperties_result result = new getNamespaceProperties_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;
+            getNamespaceProperties_result result = new getNamespaceProperties_result();
+            if (e instanceof ThriftTableOperationException) {
+              result.tope = (ThriftTableOperationException) e;
+              result.setTopeIsSet(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, getNamespaceProperties_args args, org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>> resultHandler) throws org.apache.thrift.TException {
+        iface.getNamespaceProperties(args.tinfo, args.credentials, args.ns,resultHandler);
+      }
+    }
+
+    public static class getVersionedNamespaceProperties<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getVersionedNamespaceProperties_args, TVersionedProperties> {
+      public getVersionedNamespaceProperties() {
+        super("getVersionedNamespaceProperties");
+      }
+
+      public getVersionedNamespaceProperties_args getEmptyArgsInstance() {
+        return new getVersionedNamespaceProperties_args();
+      }
+
+      public org.apache.thrift.async.AsyncMethodCallback<TVersionedProperties> 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<TVersionedProperties>() { 
+          public void onComplete(TVersionedProperties o) {
+            getVersionedNamespaceProperties_result result = new getVersionedNamespaceProperties_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;
+            getVersionedNamespaceProperties_result result = new getVersionedNamespaceProperties_result();
+            if (e instanceof ThriftTableOperationException) {
+              result.tope = (ThriftTableOperationException) e;
+              result.setTopeIsSet(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, getVersionedNamespaceProperties_args args, org.apache.thrift.async.AsyncMethodCallback<TVersionedProperties> resultHandler) throws org.apache.thrift.TException {
+        iface.getVersionedNamespaceProperties(args.tinfo, args.credentials, args.ns,resultHandler);
+      }
+    }
+
+    public static class checkClass<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, checkClass_args, java.lang.Boolean> {
+      public checkClass() {
+        super("checkClass");
+      }
+
+      public checkClass_args getEmptyArgsInstance() {
+        return new checkClass_args();
+      }
+
+      public org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean>() { 
+          public void onComplete(java.lang.Boolean o) {
+            checkClass_result result = new checkClass_result();
+            result.success = o;
+            result.setSuccessIsSet(true);
+            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;
+            checkClass_result result = new checkClass_result();
+            if (e instanceof org.apache.thrift.transport.TTransportException) {
+              _LOGGER.error("TTransportException inside handler", e);
+              fb.close();
+              return;
+            } else if (e instanceof org.apache.thrift.TApplicationException) {
+              _LOGGER.error("TApplicationException inside handler", e);
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TApplicationException)e;
+            } else {
+              _LOGGER.error("Exception inside handler", e);
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+            } catch (java.lang.Exception ex) {
+              _LOGGER.error("Exception writing to internal frame buffer", ex);
+              fb.close();
+            }
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, checkClass_args args, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
+        iface.checkClass(args.tinfo, args.credentials, args.className, args.interfaceMatch,resultHandler);
+      }
+    }
+
+    public static class checkTableClass<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, checkTableClass_args, java.lang.Boolean> {
+      public checkTableClass() {
+        super("checkTableClass");
+      }
+
+      public checkTableClass_args getEmptyArgsInstance() {
+        return new checkTableClass_args();
+      }
+
+      public org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean>() { 
+          public void onComplete(java.lang.Boolean o) {
+            checkTableClass_result result = new checkTableClass_result();
+            result.success = o;
+            result.setSuccessIsSet(true);
+            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;
+            checkTableClass_result result = new checkTableClass_result();
+            if (e instanceof ThriftSecurityException) {
+              result.sec = (ThriftSecurityException) e;
+              result.setSecIsSet(true);
+              msg = result;
+            } else if (e instanceof ThriftTableOperationException) {
+              result.tope = (ThriftTableOperationException) e;
+              result.setTopeIsSet(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, checkTableClass_args args, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
+        iface.checkTableClass(args.tinfo, args.credentials, args.tableId, args.className, args.interfaceMatch,resultHandler);
+      }
+    }
+
+    public static class checkNamespaceClass<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, checkNamespaceClass_args, java.lang.Boolean> {
+      public checkNamespaceClass() {
+        super("checkNamespaceClass");
+      }
+
+      public checkNamespaceClass_args getEmptyArgsInstance() {
+        return new checkNamespaceClass_args();
+      }
+
+      public org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean>() { 
+          public void onComplete(java.lang.Boolean o) {
+            checkNamespaceClass_result result = new checkNamespaceClass_result();
+            result.success = o;
+            result.setSuccessIsSet(true);
+            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;
+            checkNamespaceClass_result result = new checkNamespaceClass_result();
+            if (e instanceof ThriftSecurityException) {
+              result.sec = (ThriftSecurityException) e;
+              result.setSecIsSet(true);
+              msg = result;
+            } else if (e instanceof ThriftTableOperationException) {
               result.tope = (ThriftTableOperationException) e;
               result.setTopeIsSet(true);
               msg = result;
@@ -7821,13 +8787,13 @@ public class ClientService {
             case 5: // FILES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list8 = iprot.readListBegin();
-                  struct.files = new java.util.ArrayList<java.lang.String>(_list8.size);
-                  @org.apache.thrift.annotation.Nullable java.lang.String _elem9;
-                  for (int _i10 = 0; _i10 < _list8.size; ++_i10)
+                  org.apache.thrift.protocol.TList _list18 = iprot.readListBegin();
+                  struct.files = new java.util.ArrayList<java.lang.String>(_list18.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _elem19;
+                  for (int _i20 = 0; _i20 < _list18.size; ++_i20)
                   {
-                    _elem9 = iprot.readString();
-                    struct.files.add(_elem9);
+                    _elem19 = iprot.readString();
+                    struct.files.add(_elem19);
                   }
                   iprot.readListEnd();
                 }
@@ -7884,9 +8850,9 @@ public class ClientService {
           oprot.writeFieldBegin(FILES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.files.size()));
-            for (java.lang.String _iter11 : struct.files)
+            for (java.lang.String _iter21 : struct.files)
             {
-              oprot.writeString(_iter11);
+              oprot.writeString(_iter21);
             }
             oprot.writeListEnd();
           }
@@ -7960,9 +8926,9 @@ public class ClientService {
         if (struct.isSetFiles()) {
           {
             oprot.writeI32(struct.files.size());
-            for (java.lang.String _iter12 : struct.files)
+            for (java.lang.String _iter22 : struct.files)
             {
-              oprot.writeString(_iter12);
+              oprot.writeString(_iter22);
             }
           }
         }
@@ -7998,13 +8964,13 @@ public class ClientService {
         }
         if (incoming.get(4)) {
           {
-            org.apache.thrift.protocol.TList _list13 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
-            struct.files = new java.util.ArrayList<java.lang.String>(_list13.size);
-            @org.apache.thrift.annotation.Nullable java.lang.String _elem14;
-            for (int _i15 = 0; _i15 < _list13.size; ++_i15)
+            org.apache.thrift.protocol.TList _list23 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
+            struct.files = new java.util.ArrayList<java.lang.String>(_list23.size);
+            @org.apache.thrift.annotation.Nullable java.lang.String _elem24;
+            for (int _i25 = 0; _i25 < _list23.size; ++_i25)
             {
-              _elem14 = iprot.readString();
-              struct.files.add(_elem14);
+              _elem24 = iprot.readString();
+              struct.files.add(_elem24);
             }
           }
           struct.setFilesIsSet(true);
@@ -8504,13 +9470,13 @@ public class ClientService {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list16 = iprot.readListBegin();
-                  struct.success = new java.util.ArrayList<java.lang.String>(_list16.size);
-                  @org.apache.thrift.annotation.Nullable java.lang.String _elem17;
-                  for (int _i18 = 0; _i18 < _list16.size; ++_i18)
+                  org.apache.thrift.protocol.TList _list26 = iprot.readListBegin();
+                  struct.success = new java.util.ArrayList<java.lang.String>(_list26.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _elem27;
+                  for (int _i28 = 0; _i28 < _list26.size; ++_i28)
                   {
-                    _elem17 = iprot.readString();
-                    struct.success.add(_elem17);
+                    _elem27 = iprot.readString();
+                    struct.success.add(_elem27);
                   }
                   iprot.readListEnd();
                 }
@@ -8556,9 +9522,9 @@ public class ClientService {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (java.lang.String _iter19 : struct.success)
+            for (java.lang.String _iter29 : struct.success)
             {
-              oprot.writeString(_iter19);
+              oprot.writeString(_iter29);
             }
             oprot.writeListEnd();
           }
@@ -8605,9 +9571,9 @@ public class ClientService {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (java.lang.String _iter20 : struct.success)
+            for (java.lang.String _iter30 : struct.success)
             {
-              oprot.writeString(_iter20);
+              oprot.writeString(_iter30);
             }
           }
         }
@@ -8625,13 +9591,13 @@ public class ClientService {
         java.util.BitSet incoming = iprot.readBitSet(3);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list21 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
-            struct.success = new java.util.ArrayList<java.lang.String>(_list21.size);
-            @org.apache.thrift.annotation.Nullable java.lang.String _elem22;
-            for (int _i23 = 0; _i23 < _list21.size; ++_i23)
+            org.apache.thrift.protocol.TList _list31 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
+            struct.success = new java.util.ArrayList<java.lang.String>(_list31.size);
+            @org.apache.thrift.annotation.Nullable java.lang.String _elem32;
+            for (int _i33 = 0; _i33 < _list31.size; ++_i33)
             {
-              _elem22 = iprot.readString();
-              struct.success.add(_elem22);
+              _elem32 = iprot.readString();
+              struct.success.add(_elem32);
             }
           }
           struct.setSuccessIsSet(true);
@@ -10623,13 +11589,13 @@ public class ClientService {
             case 2: // TABLES
               if (schemeField.type == org.apache.thrift.protocol.TType.SET) {
                 {
-                  org.apache.thrift.protocol.TSet _set24 = iprot.readSetBegin();
-                  struct.tables = new java.util.HashSet<java.lang.String>(2*_set24.size);
-                  @org.apache.thrift.annotation.Nullable java.lang.String _elem25;
-                  for (int _i26 = 0; _i26 < _set24.size; ++_i26)
+                  org.apache.thrift.protocol.TSet _set34 = iprot.readSetBegin();
+                  struct.tables = new java.util.HashSet<java.lang.String>(2*_set34.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _elem35;
+                  for (int _i36 = 0; _i36 < _set34.size; ++_i36)
                   {
-                    _elem25 = iprot.readString();
-                    struct.tables.add(_elem25);
+                    _elem35 = iprot.readString();
+                    struct.tables.add(_elem35);
                   }
                   iprot.readSetEnd();
                 }
@@ -10671,9 +11637,9 @@ public class ClientService {
           oprot.writeFieldBegin(TABLES_FIELD_DESC);
           {
             oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRING, struct.tables.size()));
-            for (java.lang.String _iter27 : struct.tables)
+            for (java.lang.String _iter37 : struct.tables)
             {
-              oprot.writeString(_iter27);
+              oprot.writeString(_iter37);
             }
             oprot.writeSetEnd();
           }
@@ -10707,9 +11673,9 @@ public class ClientService {
         if (struct.isSetTables()) {
           {
             oprot.writeI32(struct.tables.size());
-            for (java.lang.String _iter28 : struct.tables)
+            for (java.lang.String _iter38 : struct.tables)
             {
-              oprot.writeString(_iter28);
+              oprot.writeString(_iter38);
             }
           }
         }
@@ -10724,13 +11690,13 @@ public class ClientService {
         java.util.BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TSet _set29 = iprot.readSetBegin(org.apache.thrift.protocol.TType.STRING);
-            struct.tables = new java.util.HashSet<java.lang.String>(2*_set29.size);
-            @org.apache.thrift.annotation.Nullable java.lang.String _elem30;
-            for (int _i31 = 0; _i31 < _set29.size; ++_i31)
+            org.apache.thrift.protocol.TSet _set39 = iprot.readSetBegin(org.apache.thrift.protocol.TType.STRING);
+            struct.tables = new java.util.HashSet<java.lang.String>(2*_set39.size);
+            @org.apache.thrift.annotation.Nullable java.lang.String _elem40;
+            for (int _i41 = 0; _i41 < _set39.size; ++_i41)
             {
-              _elem30 = iprot.readString();
-              struct.tables.add(_elem30);
+              _elem40 = iprot.readString();
+              struct.tables.add(_elem40);
             }
           }
           struct.setTablesIsSet(true);
@@ -11230,14 +12196,14 @@ public class ClientService {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list32 = iprot.readListBegin();
-                  struct.success = new java.util.ArrayList<TDiskUsage>(_list32.size);
-                  @org.apache.thrift.annotation.Nullable TDiskUsage _elem33;
-                  for (int _i34 = 0; _i34 < _list32.size; ++_i34)
+                  org.apache.thrift.protocol.TList _list42 = iprot.readListBegin();
+                  struct.success = new java.util.ArrayList<TDiskUsage>(_list42.size);
+                  @org.apache.thrift.annotation.Nullable TDiskUsage _elem43;
+                  for (int _i44 = 0; _i44 < _list42.size; ++_i44)
                   {
-                    _elem33 = new TDiskUsage();
-                    _elem33.read(iprot);
-                    struct.success.add(_elem33);
+                    _elem43 = new TDiskUsage();
+                    _elem43.read(iprot);
+                    struct.success.add(_elem43);
                   }
                   iprot.readListEnd();
                 }
@@ -11283,9 +12249,9 @@ public class ClientService {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (TDiskUsage _iter35 : struct.success)
+            for (TDiskUsage _iter45 : struct.success)
             {
-              _iter35.write(oprot);
+              _iter45.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -11332,9 +12298,9 @@ public class ClientService {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (TDiskUsage _iter36 : struct.success)
+            for (TDiskUsage _iter46 : struct.success)
             {
-              _iter36.write(oprot);
+              _iter46.write(oprot);
             }
           }
         }
@@ -11352,14 +12318,14 @@ public class ClientService {
         java.util.BitSet incoming = iprot.readBitSet(3);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list37 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
-            struct.success = new java.util.ArrayList<TDiskUsage>(_list37.size);
-            @org.apache.thrift.annotation.Nullable TDiskUsage _elem38;
-            for (int _i39 = 0; _i39 < _list37.size; ++_i39)
+            org.apache.thrift.protocol.TList _list47 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+            struct.success = new java.util.ArrayList<TDiskUsage>(_list47.size);
+            @org.apache.thrift.annotation.Nullable TDiskUsage _elem48;
+            for (int _i49 = 0; _i49 < _list47.size; ++_i49)
             {
-              _elem38 = new TDiskUsage();
-              _elem38.read(iprot);
-              struct.success.add(_elem38);
+              _elem48 = new TDiskUsage();
+              _elem48.read(iprot);
+              struct.success.add(_elem48);
             }
           }
           struct.setSuccessIsSet(true);
@@ -12259,13 +13225,13 @@ public class ClientService {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.SET) {
                 {
-                  org.apache.thrift.protocol.TSet _set40 = iprot.readSetBegin();
-                  struct.success = new java.util.HashSet<java.lang.String>(2*_set40.size);
-                  @org.apache.thrift.annotation.Nullable java.lang.String _elem41;
-                  for (int _i42 = 0; _i42 < _set40.size; ++_i42)
+                  org.apache.thrift.protocol.TSet _set50 = iprot.readSetBegin();
+                  struct.success = new java.util.HashSet<java.lang.String>(2*_set50.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _elem51;
+                  for (int _i52 = 0; _i52 < _set50.size; ++_i52)
                   {
-                    _elem41 = iprot.readString();
-                    struct.success.add(_elem41);
+                    _elem51 = iprot.readString();
+                    struct.success.add(_elem51);
                   }
                   iprot.readSetEnd();
                 }
@@ -12302,9 +13268,9 @@ public class ClientService {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (java.lang.String _iter43 : struct.success)
+            for (java.lang.String _iter53 : struct.success)
             {
-              oprot.writeString(_iter43);
+              oprot.writeString(_iter53);
             }
             oprot.writeSetEnd();
           }
@@ -12343,9 +13309,9 @@ public class ClientService {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (java.lang.String _iter44 : struct.success)
+            for (java.lang.String _iter54 : struct.success)
             {
-              oprot.writeString(_iter44);
+              oprot.writeString(_iter54);
             }
           }
         }
@@ -12360,13 +13326,13 @@ public class ClientService {
         java.util.BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TSet _set45 = iprot.readSetBegin(org.apache.thrift.protocol.TType.STRING);
-            struct.success = new java.util.HashSet<java.lang.String>(2*_set45.size);
-            @org.apache.thrift.annotation.Nullable java.lang.String _elem46;
-            for (int _i47 = 0; _i47 < _set45.size; ++_i47)
+            org.apache.thrift.protocol.TSet _set55 = iprot.readSetBegin(org.apache.thrift.protocol.TType.STRING);
+            struct.success = new java.util.HashSet<java.lang.String>(2*_set55.size);
+            @org.apache.thrift.annotation.Nullable java.lang.String _elem56;
+            for (int _i57 = 0; _i57 < _set55.size; ++_i57)
             {
-              _elem46 = iprot.readString();
-              struct.success.add(_elem46);
+              _elem56 = iprot.readString();
+              struct.success.add(_elem56);
             }
           }
           struct.setSuccessIsSet(true);
@@ -18077,13 +19043,13 @@ public class ClientService {
             case 3: // AUTHORIZATIONS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list48 = iprot.readListBegin();
-                  struct.authorizations = new java.util.ArrayList<java.nio.ByteBuffer>(_list48.size);
-                  @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem49;
-                  for (int _i50 = 0; _i50 < _list48.size; ++_i50)
+                  org.apache.thrift.protocol.TList _list58 = iprot.readListBegin();
+                  struct.authorizations = new java.util.ArrayList<java.nio.ByteBuffer>(_list58.size);
+                  @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem59;
+                  for (int _i60 = 0; _i60 < _list58.size; ++_i60)
                   {
-                    _elem49 = iprot.readBinary();
-                    struct.authorizations.add(_elem49);
+                    _elem59 = iprot.readBinary();
+                    struct.authorizations.add(_elem59);
                   }
                   iprot.readListEnd();
                 }
@@ -18116,9 +19082,9 @@ public class ClientService {
           oprot.writeFieldBegin(AUTHORIZATIONS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.authorizations.size()));
-            for (java.nio.ByteBuffer _iter51 : struct.authorizations)
+            for (java.nio.ByteBuffer _iter61 : struct.authorizations)
             {
-              oprot.writeBinary(_iter51);
+              oprot.writeBinary(_iter61);
             }
             oprot.writeListEnd();
           }
@@ -18177,9 +19143,9 @@ public class ClientService {
         if (struct.isSetAuthorizations()) {
           {
             oprot.writeI32(struct.authorizations.size());
-            for (java.nio.ByteBuffer _iter52 : struct.authorizations)
+            for (java.nio.ByteBuffer _iter62 : struct.authorizations)
             {
-              oprot.writeBinary(_iter52);
+              oprot.writeBinary(_iter62);
             }
           }
         }
@@ -18205,13 +19171,13 @@ public class ClientService {
         }
         if (incoming.get(3)) {
           {
-            org.apache.thrift.protocol.TList _list53 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
-            struct.authorizations = new java.util.ArrayList<java.nio.ByteBuffer>(_list53.size);
-            @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem54;
-            for (int _i55 = 0; _i55 < _list53.size; ++_i55)
+            org.apache.thrift.protocol.TList _list63 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
+            struct.authorizations = new java.util.ArrayList<java.nio.ByteBuffer>(_list63.size);
+            @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem64;
+            for (int _i65 = 0; _i65 < _list63.size; ++_i65)
             {
-              _elem54 = iprot.readBinary();
-              struct.authorizations.add(_elem54);
+              _elem64 = iprot.readBinary();
+              struct.authorizations.add(_elem64);
             }
           }
           struct.setAuthorizationsIsSet(true);
@@ -19573,13 +20539,13 @@ public class ClientService {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list56 = iprot.readListBegin();
-                  struct.success = new java.util.ArrayList<java.nio.ByteBuffer>(_list56.size);
-                  @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem57;
-                  for (int _i58 = 0; _i58 < _list56.size; ++_i58)
+                  org.apache.thrift.protocol.TList _list66 = iprot.readListBegin();
+                  struct.success = new java.util.ArrayList<java.nio.ByteBuffer>(_list66.size);
+                  @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem67;
+                  for (int _i68 = 0; _i68 < _list66.size; ++_i68)
                   {
-                    _elem57 = iprot.readBinary();
-                    struct.success.add(_elem57);
+                    _elem67 = iprot.readBinary();
+                    struct.success.add(_elem67);
                   }
                   iprot.readListEnd();
                 }
@@ -19616,9 +20582,9 @@ public class ClientService {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (java.nio.ByteBuffer _iter59 : struct.success)
+            for (java.nio.ByteBuffer _iter69 : struct.success)
             {
-              oprot.writeBinary(_iter59);
+              oprot.writeBinary(_iter69);
             }
             oprot.writeListEnd();
           }
@@ -19657,9 +20623,9 @@ public class ClientService {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (java.nio.ByteBuffer _iter60 : struct.success)
+            for (java.nio.ByteBuffer _iter70 : struct.success)
             {
-              oprot.writeBinary(_iter60);
+              oprot.writeBinary(_iter70);
             }
           }
         }
@@ -19674,13 +20640,13 @@ public class ClientService {
         java.util.BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list61 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
-            struct.success = new java.util.ArrayList<java.nio.ByteBuffer>(_list61.size);
-            @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem62;
-            for (int _i63 = 0; _i63 < _list61.size; ++_i63)
+            org.apache.thrift.protocol.TList _list71 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
+            struct.success = new java.util.ArrayList<java.nio.ByteBuffer>(_list71.size);
+            @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem72;
+            for (int _i73 = 0; _i73 < _list71.size; ++_i73)
             {
-              _elem62 = iprot.readBinary();
-              struct.success.add(_elem62);
+              _elem72 = iprot.readBinary();
+              struct.success.add(_elem72);
             }
           }
           struct.setSuccessIsSet(true);
@@ -30349,70 +31315,6112 @@ public class ClientService {
     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.SEC, new org.apache.thrift.meta_data.FieldMetaData("sec", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ThriftSecurityException.class)));
+      tmpMap.put(_Fields.SEC, new org.apache.thrift.meta_data.FieldMetaData("sec", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ThriftSecurityException.class)));
+      tmpMap.put(_Fields.TOPE, new org.apache.thrift.meta_data.FieldMetaData("tope", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ThriftTableOperationException.class)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(revokeNamespacePermission_result.class, metaDataMap);
+    }
+
+    public revokeNamespacePermission_result() {
+    }
+
+    public revokeNamespacePermission_result(
+      ThriftSecurityException sec,
+      ThriftTableOperationException tope)
+    {
+      this();
+      this.sec = sec;
+      this.tope = tope;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public revokeNamespacePermission_result(revokeNamespacePermission_result other) {
+      if (other.isSetSec()) {
+        this.sec = new ThriftSecurityException(other.sec);
+      }
+      if (other.isSetTope()) {
+        this.tope = new ThriftTableOperationException(other.tope);
+      }
+    }
+
+    public revokeNamespacePermission_result deepCopy() {
+      return new revokeNamespacePermission_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.sec = null;
+      this.tope = null;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public ThriftSecurityException getSec() {
+      return this.sec;
+    }
+
+    public revokeNamespacePermission_result setSec(@org.apache.thrift.annotation.Nullable ThriftSecurityException sec) {
+      this.sec = sec;
+      return this;
+    }
+
+    public void unsetSec() {
+      this.sec = null;
+    }
+
+    /** Returns true if field sec is set (has been assigned a value) and false otherwise */
+    public boolean isSetSec() {
+      return this.sec != null;
+    }
+
+    public void setSecIsSet(boolean value) {
+      if (!value) {
+        this.sec = null;
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public ThriftTableOperationException getTope() {
+      return this.tope;
+    }
+
+    public revokeNamespacePermission_result setTope(@org.apache.thrift.annotation.Nullable ThriftTableOperationException tope) {
+      this.tope = tope;
+      return this;
+    }
+
+    public void unsetTope() {
+      this.tope = null;
+    }
+
+    /** Returns true if field tope is set (has been assigned a value) and false otherwise */
+    public boolean isSetTope() {
+      return this.tope != null;
+    }
+
+    public void setTopeIsSet(boolean value) {
+      if (!value) {
+        this.tope = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+      switch (field) {
+      case SEC:
+        if (value == null) {
+          unsetSec();
+        } else {
+          setSec((ThriftSecurityException)value);
+        }
+        break;
+
+      case TOPE:
+        if (value == null) {
+          unsetTope();
+        } else {
+          setTope((ThriftTableOperationException)value);
+        }
+        break;
+
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case SEC:
+        return getSec();
+
+      case TOPE:
+        return getTope();
+
+      }
+      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 SEC:
+        return isSetSec();
+      case TOPE:
+        return isSetTope();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that instanceof revokeNamespacePermission_result)
+        return this.equals((revokeNamespacePermission_result)that);
+      return false;
+    }
+
+    public boolean equals(revokeNamespacePermission_result that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return true;
+
+      boolean this_present_sec = true && this.isSetSec();
+      boolean that_present_sec = true && that.isSetSec();
+      if (this_present_sec || that_present_sec) {
+        if (!(this_present_sec && that_present_sec))
+          return false;
+        if (!this.sec.equals(that.sec))
+          return false;
+      }
+
+      boolean this_present_tope = true && this.isSetTope();
+      boolean that_present_tope = true && that.isSetTope();
+      if (this_present_tope || that_present_tope) {
+        if (!(this_present_tope && that_present_tope))
+          return false;
+        if (!this.tope.equals(that.tope))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      hashCode = hashCode * 8191 + ((isSetSec()) ? 131071 : 524287);
+      if (isSetSec())
+        hashCode = hashCode * 8191 + sec.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetTope()) ? 131071 : 524287);
+      if (isSetTope())
+        hashCode = hashCode * 8191 + tope.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(revokeNamespacePermission_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = java.lang.Boolean.compare(isSetSec(), other.isSetSec());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetSec()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sec, other.sec);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.compare(isSetTope(), other.isSetTope());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetTope()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tope, other.tope);
+        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("revokeNamespacePermission_result(");
+      boolean first = true;
+
+      sb.append("sec:");
+      if (this.sec == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.sec);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("tope:");
+      if (this.tope == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.tope);
+      }
+      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 revokeNamespacePermission_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public revokeNamespacePermission_resultStandardScheme getScheme() {
+        return new revokeNamespacePermission_resultStandardScheme();
+      }
+    }
+
+    private static class revokeNamespacePermission_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<revokeNamespacePermission_result> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, revokeNamespacePermission_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: // SEC
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.sec = new ThriftSecurityException();
+                struct.sec.read(iprot);
+                struct.setSecIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 2: // TOPE
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.tope = new ThriftTableOperationException();
+                struct.tope.read(iprot);
+                struct.setTopeIsSet(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, revokeNamespacePermission_result struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.sec != null) {
+          oprot.writeFieldBegin(SEC_FIELD_DESC);
+          struct.sec.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.tope != null) {
+          oprot.writeFieldBegin(TOPE_FIELD_DESC);
+          struct.tope.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class revokeNamespacePermission_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public revokeNamespacePermission_resultTupleScheme getScheme() {
+        return new revokeNamespacePermission_resultTupleScheme();
+      }
+    }
+
+    private static class revokeNamespacePermission_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<revokeNamespacePermission_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, revokeNamespacePermission_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.isSetSec()) {
+          optionals.set(0);
+        }
+        if (struct.isSetTope()) {
+          optionals.set(1);
+        }
+        oprot.writeBitSet(optionals, 2);
+        if (struct.isSetSec()) {
+          struct.sec.write(oprot);
+        }
+        if (struct.isSetTope()) {
+          struct.tope.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, revokeNamespacePermission_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.sec = new ThriftSecurityException();
+          struct.sec.read(iprot);
+          struct.setSecIsSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.tope = new ThriftTableOperationException();
+          struct.tope.read(iprot);
+          struct.setTopeIsSet(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 getConfiguration_args implements org.apache.thrift.TBase<getConfiguration_args, getConfiguration_args._Fields>, java.io.Serializable, Cloneable, Comparable<getConfiguration_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getConfiguration_args");
+
+    private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+    private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)3);
+    private static final org.apache.thrift.protocol.TField TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("type", org.apache.thrift.protocol.TType.I32, (short)1);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getConfiguration_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getConfiguration_argsTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
+    /**
+     * 
+     * @see ConfigurationType
+     */
+    public @org.apache.thrift.annotation.Nullable ConfigurationType type; // 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 {
+      TINFO((short)2, "tinfo"),
+      CREDENTIALS((short)3, "credentials"),
+      /**
+       * 
+       * @see ConfigurationType
+       */
+      TYPE((short)1, "type");
+
+      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 2: // TINFO
+            return TINFO;
+          case 3: // CREDENTIALS
+            return CREDENTIALS;
+          case 1: // TYPE
+            return TYPE;
+          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.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+      tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
+      tmpMap.put(_Fields.TYPE, new org.apache.thrift.meta_data.FieldMetaData("type", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, ConfigurationType.class)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getConfiguration_args.class, metaDataMap);
+    }
+
+    public getConfiguration_args() {
+    }
+
+    public getConfiguration_args(
+      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
+      ConfigurationType type)
+    {
+      this();
+      this.tinfo = tinfo;
+      this.credentials = credentials;
+      this.type = type;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public getConfiguration_args(getConfiguration_args other) {
+      if (other.isSetTinfo()) {
+        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+      }
+      if (other.isSetCredentials()) {
+        this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
+      }
+      if (other.isSetType()) {
+        this.type = other.type;
+      }
+    }
+
+    public getConfiguration_args deepCopy() {
+      return new getConfiguration_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.tinfo = null;
+      this.credentials = null;
+      this.type = null;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+      return this.tinfo;
+    }
+
+    public getConfiguration_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+      this.tinfo = tinfo;
+      return this;
+    }
+
+    public void unsetTinfo() {
+      this.tinfo = null;
+    }
+
+    /** Returns true if field tinfo is set (has been assigned a value) and false otherwise */
+    public boolean isSetTinfo() {
+      return this.tinfo != null;
+    }
+
+    public void setTinfoIsSet(boolean value) {
+      if (!value) {
+        this.tinfo = null;
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.securityImpl.thrift.TCredentials getCredentials() {
+      return this.credentials;
+    }
+
+    public getConfiguration_args setCredentials(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) {
+      this.credentials = credentials;
+      return this;
+    }
+
+    public void unsetCredentials() {
+      this.credentials = null;
+    }
+
+    /** Returns true if field credentials is set (has been assigned a value) and false otherwise */
+    public boolean isSetCredentials() {
+      return this.credentials != null;
+    }
+
+    public void setCredentialsIsSet(boolean value) {
+      if (!value) {
+        this.credentials = null;
+      }
+    }
+
+    /**
+     * 
+     * @see ConfigurationType
+     */
+    @org.apache.thrift.annotation.Nullable
+    public ConfigurationType getType() {
+      return this.type;
+    }
+
+    /**
+     * 
+     * @see ConfigurationType
+     */
+    public getConfiguration_args setType(@org.apache.thrift.annotation.Nullable ConfigurationType type) {
+      this.type = type;
+      return this;
+    }
+
+    public void unsetType() {
+      this.type = null;
+    }
+
+    /** Returns true if field type is set (has been assigned a value) and false otherwise */
+    public boolean isSetType() {
+      return this.type != null;
+    }
+
+    public void setTypeIsSet(boolean value) {
+      if (!value) {
+        this.type = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+      switch (field) {
+      case TINFO:
+        if (value == null) {
+          unsetTinfo();
+        } else {
+          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+        }
+        break;
+
+      case CREDENTIALS:
+        if (value == null) {
+          unsetCredentials();
+        } else {
+          setCredentials((org.apache.accumulo.core.securityImpl.thrift.TCredentials)value);
+        }
+        break;
+
+      case TYPE:
+        if (value == null) {
+          unsetType();
+        } else {
+          setType((ConfigurationType)value);
+        }
+        break;
+
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case TINFO:
+        return getTinfo();
+
+      case CREDENTIALS:
+        return getCredentials();
+
+      case TYPE:
+        return getType();
+
+      }
+      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 TINFO:
+        return isSetTinfo();
+      case CREDENTIALS:
+        return isSetCredentials();
+      case TYPE:
+        return isSetType();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that instanceof getConfiguration_args)
+        return this.equals((getConfiguration_args)that);
+      return false;
+    }
+
+    public boolean equals(getConfiguration_args that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return true;
+
+      boolean this_present_tinfo = true && this.isSetTinfo();
+      boolean that_present_tinfo = true && that.isSetTinfo();
+      if (this_present_tinfo || that_present_tinfo) {
+        if (!(this_present_tinfo && that_present_tinfo))
+          return false;
+        if (!this.tinfo.equals(that.tinfo))
+          return false;
+      }
+
+      boolean this_present_credentials = true && this.isSetCredentials();
+      boolean that_present_credentials = true && that.isSetCredentials();
+      if (this_present_credentials || that_present_credentials) {
+        if (!(this_present_credentials && that_present_credentials))
+          return false;
+        if (!this.credentials.equals(that.credentials))
+          return false;
+      }
+
+      boolean this_present_type = true && this.isSetType();
+      boolean that_present_type = true && that.isSetType();
+      if (this_present_type || that_present_type) {
+        if (!(this_present_type && that_present_type))
+          return false;
+        if (!this.type.equals(that.type))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      hashCode = hashCode * 8191 + ((isSetTinfo()) ? 131071 : 524287);
+      if (isSetTinfo())
+        hashCode = hashCode * 8191 + tinfo.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetCredentials()) ? 131071 : 524287);
+      if (isSetCredentials())
+        hashCode = hashCode * 8191 + credentials.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetType()) ? 131071 : 524287);
+      if (isSetType())
+        hashCode = hashCode * 8191 + type.getValue();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(getConfiguration_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = java.lang.Boolean.compare(isSetTinfo(), other.isSetTinfo());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetTinfo()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tinfo, other.tinfo);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.compare(isSetCredentials(), other.isSetCredentials());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetCredentials()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.credentials, other.credentials);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.compare(isSetType(), other.isSetType());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetType()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.type, other.type);
+        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("getConfiguration_args(");
+      boolean first = true;
+
+      sb.append("tinfo:");
+      if (this.tinfo == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.tinfo);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("credentials:");
+      if (this.credentials == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.credentials);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("type:");
+      if (this.type == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.type);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+      if (tinfo != null) {
+        tinfo.validate();
+      }
+      if (credentials != null) {
+        credentials.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 getConfiguration_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getConfiguration_argsStandardScheme getScheme() {
+        return new getConfiguration_argsStandardScheme();
+      }
+    }
+
+    private static class getConfiguration_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<getConfiguration_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, getConfiguration_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 2: // TINFO
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+                struct.tinfo.read(iprot);
+                struct.setTinfoIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 3: // CREDENTIALS
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
+                struct.credentials.read(iprot);
+                struct.setCredentialsIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 1: // TYPE
+              if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+                struct.type = org.apache.accumulo.core.clientImpl.thrift.ConfigurationType.findByValue(iprot.readI32());
+                struct.setTypeIsSet(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, getConfiguration_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.type != null) {
+          oprot.writeFieldBegin(TYPE_FIELD_DESC);
+          oprot.writeI32(struct.type.getValue());
+          oprot.writeFieldEnd();
+        }
+        if (struct.tinfo != null) {
+          oprot.writeFieldBegin(TINFO_FIELD_DESC);
+          struct.tinfo.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.credentials != null) {
+          oprot.writeFieldBegin(CREDENTIALS_FIELD_DESC);
+          struct.credentials.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class getConfiguration_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getConfiguration_argsTupleScheme getScheme() {
+        return new getConfiguration_argsTupleScheme();
+      }
+    }
+
+    private static class getConfiguration_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<getConfiguration_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, getConfiguration_args 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.isSetTinfo()) {
+          optionals.set(0);
+        }
+        if (struct.isSetCredentials()) {
+          optionals.set(1);
+        }
+        if (struct.isSetType()) {
+          optionals.set(2);
+        }
+        oprot.writeBitSet(optionals, 3);
+        if (struct.isSetTinfo()) {
+          struct.tinfo.write(oprot);
+        }
+        if (struct.isSetCredentials()) {
+          struct.credentials.write(oprot);
+        }
+        if (struct.isSetType()) {
+          oprot.writeI32(struct.type.getValue());
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, getConfiguration_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet incoming = iprot.readBitSet(3);
+        if (incoming.get(0)) {
+          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+          struct.tinfo.read(iprot);
+          struct.setTinfoIsSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
+          struct.credentials.read(iprot);
+          struct.setCredentialsIsSet(true);
+        }
+        if (incoming.get(2)) {
+          struct.type = org.apache.accumulo.core.clientImpl.thrift.ConfigurationType.findByValue(iprot.readI32());
+          struct.setTypeIsSet(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 getConfiguration_result implements org.apache.thrift.TBase<getConfiguration_result, getConfiguration_result._Fields>, java.io.Serializable, Cloneable, Comparable<getConfiguration_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getConfiguration_result");
+
+    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.MAP, (short)0);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getConfiguration_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getConfiguration_resultTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,java.lang.String> success; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      SUCCESS((short)0, "success");
+
+      private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
+
+      static {
+        for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      @org.apache.thrift.annotation.Nullable
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 0: // SUCCESS
+            return SUCCESS;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      @org.apache.thrift.annotation.Nullable
+      public static _Fields findByName(java.lang.String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final java.lang.String _fieldName;
+
+      _Fields(short thriftId, java.lang.String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public java.lang.String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+              new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+              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(getConfiguration_result.class, metaDataMap);
+    }
+
+    public getConfiguration_result() {
+    }
+
+    public getConfiguration_result(
+      java.util.Map<java.lang.String,java.lang.String> success)
+    {
+      this();
+      this.success = success;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public getConfiguration_result(getConfiguration_result other) {
+      if (other.isSetSuccess()) {
+        java.util.Map<java.lang.String,java.lang.String> __this__success = new java.util.HashMap<java.lang.String,java.lang.String>(other.success);
+        this.success = __this__success;
+      }
+    }
+
+    public getConfiguration_result deepCopy() {
+      return new getConfiguration_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.success = null;
+    }
+
+    public int getSuccessSize() {
+      return (this.success == null) ? 0 : this.success.size();
+    }
+
+    public void putToSuccess(java.lang.String key, java.lang.String val) {
+      if (this.success == null) {
+        this.success = new java.util.HashMap<java.lang.String,java.lang.String>();
+      }
+      this.success.put(key, val);
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.util.Map<java.lang.String,java.lang.String> getSuccess() {
+      return this.success;
+    }
+
+    public getConfiguration_result setSuccess(@org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,java.lang.String> success) {
+      this.success = success;
+      return this;
+    }
+
+    public void unsetSuccess() {
+      this.success = null;
+    }
+
+    /** Returns true if field success is set (has been assigned a value) and false otherwise */
+    public boolean isSetSuccess() {
+      return this.success != null;
+    }
+
+    public void setSuccessIsSet(boolean value) {
+      if (!value) {
+        this.success = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+      switch (field) {
+      case SUCCESS:
+        if (value == null) {
+          unsetSuccess();
+        } else {
+          setSuccess((java.util.Map<java.lang.String,java.lang.String>)value);
+        }
+        break;
+
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return getSuccess();
+
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case SUCCESS:
+        return isSetSuccess();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that instanceof getConfiguration_result)
+        return this.equals((getConfiguration_result)that);
+      return false;
+    }
+
+    public boolean equals(getConfiguration_result that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return true;
+
+      boolean this_present_success = true && this.isSetSuccess();
+      boolean that_present_success = true && that.isSetSuccess();
+      if (this_present_success || that_present_success) {
+        if (!(this_present_success && that_present_success))
+          return false;
+        if (!this.success.equals(that.success))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      hashCode = hashCode * 8191 + ((isSetSuccess()) ? 131071 : 524287);
+      if (isSetSuccess())
+        hashCode = hashCode * 8191 + success.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(getConfiguration_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = java.lang.Boolean.compare(isSetSuccess(), other.isSetSuccess());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetSuccess()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      scheme(iprot).read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      scheme(oprot).write(oprot, this);
+      }
+
+    @Override
+    public java.lang.String toString() {
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("getConfiguration_result(");
+      boolean first = true;
+
+      sb.append("success:");
+      if (this.success == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.success);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class getConfiguration_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getConfiguration_resultStandardScheme getScheme() {
+        return new getConfiguration_resultStandardScheme();
+      }
+    }
+
+    private static class getConfiguration_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<getConfiguration_result> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, getConfiguration_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 0: // SUCCESS
+              if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+                {
+                  org.apache.thrift.protocol.TMap _map74 = iprot.readMapBegin();
+                  struct.success = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map74.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _key75;
+                  @org.apache.thrift.annotation.Nullable java.lang.String _val76;
+                  for (int _i77 = 0; _i77 < _map74.size; ++_i77)
+                  {
+                    _key75 = iprot.readString();
+                    _val76 = iprot.readString();
+                    struct.success.put(_key75, _val76);
+                  }
+                  iprot.readMapEnd();
+                }
+                struct.setSuccessIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+
+        // check for required fields of primitive type, which can't be checked in the validate method
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, getConfiguration_result struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.success != null) {
+          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+          {
+            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.success.size()));
+            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter78 : struct.success.entrySet())
+            {
+              oprot.writeString(_iter78.getKey());
+              oprot.writeString(_iter78.getValue());
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class getConfiguration_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getConfiguration_resultTupleScheme getScheme() {
+        return new getConfiguration_resultTupleScheme();
+      }
+    }
+
+    private static class getConfiguration_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<getConfiguration_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, getConfiguration_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet optionals = new java.util.BitSet();
+        if (struct.isSetSuccess()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetSuccess()) {
+          {
+            oprot.writeI32(struct.success.size());
+            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter79 : struct.success.entrySet())
+            {
+              oprot.writeString(_iter79.getKey());
+              oprot.writeString(_iter79.getValue());
+            }
+          }
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, getConfiguration_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)) {
+          {
+            org.apache.thrift.protocol.TMap _map80 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); 
+            struct.success = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map80.size);
+            @org.apache.thrift.annotation.Nullable java.lang.String _key81;
+            @org.apache.thrift.annotation.Nullable java.lang.String _val82;
+            for (int _i83 = 0; _i83 < _map80.size; ++_i83)
+            {
+              _key81 = iprot.readString();
+              _val82 = iprot.readString();
+              struct.success.put(_key81, _val82);
+            }
+          }
+          struct.setSuccessIsSet(true);
+        }
+      }
+    }
+
+    private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
+      return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
+    }
+  }
+
+  public static class getSystemProperties_args implements org.apache.thrift.TBase<getSystemProperties_args, getSystemProperties_args._Fields>, java.io.Serializable, Cloneable, Comparable<getSystemProperties_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getSystemProperties_args");
+
+    private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+    private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getSystemProperties_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getSystemProperties_argsTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // 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 {
+      TINFO((short)1, "tinfo"),
+      CREDENTIALS((short)2, "credentials");
+
+      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: // TINFO
+            return TINFO;
+          case 2: // CREDENTIALS
+            return CREDENTIALS;
+          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.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+      tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getSystemProperties_args.class, metaDataMap);
+    }
+
+    public getSystemProperties_args() {
+    }
+
+    public getSystemProperties_args(
+      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials)
+    {
+      this();
+      this.tinfo = tinfo;
+      this.credentials = credentials;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public getSystemProperties_args(getSystemProperties_args other) {
+      if (other.isSetTinfo()) {
+        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+      }
+      if (other.isSetCredentials()) {
+        this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
+      }
+    }
+
+    public getSystemProperties_args deepCopy() {
+      return new getSystemProperties_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.tinfo = null;
+      this.credentials = null;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+      return this.tinfo;
+    }
+
+    public getSystemProperties_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+      this.tinfo = tinfo;
+      return this;
+    }
+
+    public void unsetTinfo() {
+      this.tinfo = null;
+    }
+
+    /** Returns true if field tinfo is set (has been assigned a value) and false otherwise */
+    public boolean isSetTinfo() {
+      return this.tinfo != null;
+    }
+
+    public void setTinfoIsSet(boolean value) {
+      if (!value) {
+        this.tinfo = null;
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.securityImpl.thrift.TCredentials getCredentials() {
+      return this.credentials;
+    }
+
+    public getSystemProperties_args setCredentials(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) {
+      this.credentials = credentials;
+      return this;
+    }
+
+    public void unsetCredentials() {
+      this.credentials = null;
+    }
+
+    /** Returns true if field credentials is set (has been assigned a value) and false otherwise */
+    public boolean isSetCredentials() {
+      return this.credentials != null;
+    }
+
+    public void setCredentialsIsSet(boolean value) {
+      if (!value) {
+        this.credentials = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+      switch (field) {
+      case TINFO:
+        if (value == null) {
+          unsetTinfo();
+        } else {
+          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+        }
+        break;
+
+      case CREDENTIALS:
+        if (value == null) {
+          unsetCredentials();
+        } else {
+          setCredentials((org.apache.accumulo.core.securityImpl.thrift.TCredentials)value);
+        }
+        break;
+
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case TINFO:
+        return getTinfo();
+
+      case CREDENTIALS:
+        return getCredentials();
+
+      }
+      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 TINFO:
+        return isSetTinfo();
+      case CREDENTIALS:
+        return isSetCredentials();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that instanceof getSystemProperties_args)
+        return this.equals((getSystemProperties_args)that);
+      return false;
+    }
+
+    public boolean equals(getSystemProperties_args that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return true;
+
+      boolean this_present_tinfo = true && this.isSetTinfo();
+      boolean that_present_tinfo = true && that.isSetTinfo();
+      if (this_present_tinfo || that_present_tinfo) {
+        if (!(this_present_tinfo && that_present_tinfo))
+          return false;
+        if (!this.tinfo.equals(that.tinfo))
+          return false;
+      }
+
+      boolean this_present_credentials = true && this.isSetCredentials();
+      boolean that_present_credentials = true && that.isSetCredentials();
+      if (this_present_credentials || that_present_credentials) {
+        if (!(this_present_credentials && that_present_credentials))
+          return false;
+        if (!this.credentials.equals(that.credentials))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      hashCode = hashCode * 8191 + ((isSetTinfo()) ? 131071 : 524287);
+      if (isSetTinfo())
+        hashCode = hashCode * 8191 + tinfo.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetCredentials()) ? 131071 : 524287);
+      if (isSetCredentials())
+        hashCode = hashCode * 8191 + credentials.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(getSystemProperties_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = java.lang.Boolean.compare(isSetTinfo(), other.isSetTinfo());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetTinfo()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tinfo, other.tinfo);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.compare(isSetCredentials(), other.isSetCredentials());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetCredentials()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.credentials, other.credentials);
+        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("getSystemProperties_args(");
+      boolean first = true;
+
+      sb.append("tinfo:");
+      if (this.tinfo == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.tinfo);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("credentials:");
+      if (this.credentials == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.credentials);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+      if (tinfo != null) {
+        tinfo.validate();
+      }
+      if (credentials != null) {
+        credentials.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 getSystemProperties_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getSystemProperties_argsStandardScheme getScheme() {
+        return new getSystemProperties_argsStandardScheme();
+      }
+    }
+
+    private static class getSystemProperties_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<getSystemProperties_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, getSystemProperties_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: // TINFO
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+                struct.tinfo.read(iprot);
+                struct.setTinfoIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 2: // CREDENTIALS
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
+                struct.credentials.read(iprot);
+                struct.setCredentialsIsSet(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, getSystemProperties_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.tinfo != null) {
+          oprot.writeFieldBegin(TINFO_FIELD_DESC);
+          struct.tinfo.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.credentials != null) {
+          oprot.writeFieldBegin(CREDENTIALS_FIELD_DESC);
+          struct.credentials.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class getSystemProperties_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getSystemProperties_argsTupleScheme getScheme() {
+        return new getSystemProperties_argsTupleScheme();
+      }
+    }
+
+    private static class getSystemProperties_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<getSystemProperties_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, getSystemProperties_args 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.isSetTinfo()) {
+          optionals.set(0);
+        }
+        if (struct.isSetCredentials()) {
+          optionals.set(1);
+        }
+        oprot.writeBitSet(optionals, 2);
+        if (struct.isSetTinfo()) {
+          struct.tinfo.write(oprot);
+        }
+        if (struct.isSetCredentials()) {
+          struct.credentials.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, getSystemProperties_args 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.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+          struct.tinfo.read(iprot);
+          struct.setTinfoIsSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
+          struct.credentials.read(iprot);
+          struct.setCredentialsIsSet(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 getSystemProperties_result implements org.apache.thrift.TBase<getSystemProperties_result, getSystemProperties_result._Fields>, java.io.Serializable, Cloneable, Comparable<getSystemProperties_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getSystemProperties_result");
+
+    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.MAP, (short)0);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getSystemProperties_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getSystemProperties_resultTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,java.lang.String> success; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      SUCCESS((short)0, "success");
+
+      private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
+
+      static {
+        for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      @org.apache.thrift.annotation.Nullable
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 0: // SUCCESS
+            return SUCCESS;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      @org.apache.thrift.annotation.Nullable
+      public static _Fields findByName(java.lang.String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final java.lang.String _fieldName;
+
+      _Fields(short thriftId, java.lang.String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public java.lang.String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+              new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+              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(getSystemProperties_result.class, metaDataMap);
+    }
+
+    public getSystemProperties_result() {
+    }
+
+    public getSystemProperties_result(
+      java.util.Map<java.lang.String,java.lang.String> success)
+    {
+      this();
+      this.success = success;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public getSystemProperties_result(getSystemProperties_result other) {
+      if (other.isSetSuccess()) {
+        java.util.Map<java.lang.String,java.lang.String> __this__success = new java.util.HashMap<java.lang.String,java.lang.String>(other.success);
+        this.success = __this__success;
+      }
+    }
+
+    public getSystemProperties_result deepCopy() {
+      return new getSystemProperties_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.success = null;
+    }
+
+    public int getSuccessSize() {
+      return (this.success == null) ? 0 : this.success.size();
+    }
+
+    public void putToSuccess(java.lang.String key, java.lang.String val) {
+      if (this.success == null) {
+        this.success = new java.util.HashMap<java.lang.String,java.lang.String>();
+      }
+      this.success.put(key, val);
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.util.Map<java.lang.String,java.lang.String> getSuccess() {
+      return this.success;
+    }
+
+    public getSystemProperties_result setSuccess(@org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,java.lang.String> success) {
+      this.success = success;
+      return this;
+    }
+
+    public void unsetSuccess() {
+      this.success = null;
+    }
+
+    /** Returns true if field success is set (has been assigned a value) and false otherwise */
+    public boolean isSetSuccess() {
+      return this.success != null;
+    }
+
+    public void setSuccessIsSet(boolean value) {
+      if (!value) {
+        this.success = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+      switch (field) {
+      case SUCCESS:
+        if (value == null) {
+          unsetSuccess();
+        } else {
+          setSuccess((java.util.Map<java.lang.String,java.lang.String>)value);
+        }
+        break;
+
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return getSuccess();
+
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case SUCCESS:
+        return isSetSuccess();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that instanceof getSystemProperties_result)
+        return this.equals((getSystemProperties_result)that);
+      return false;
+    }
+
+    public boolean equals(getSystemProperties_result that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return true;
+
+      boolean this_present_success = true && this.isSetSuccess();
+      boolean that_present_success = true && that.isSetSuccess();
+      if (this_present_success || that_present_success) {
+        if (!(this_present_success && that_present_success))
+          return false;
+        if (!this.success.equals(that.success))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      hashCode = hashCode * 8191 + ((isSetSuccess()) ? 131071 : 524287);
+      if (isSetSuccess())
+        hashCode = hashCode * 8191 + success.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(getSystemProperties_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = java.lang.Boolean.compare(isSetSuccess(), other.isSetSuccess());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetSuccess()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      scheme(iprot).read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      scheme(oprot).write(oprot, this);
+      }
+
+    @Override
+    public java.lang.String toString() {
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("getSystemProperties_result(");
+      boolean first = true;
+
+      sb.append("success:");
+      if (this.success == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.success);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class getSystemProperties_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getSystemProperties_resultStandardScheme getScheme() {
+        return new getSystemProperties_resultStandardScheme();
+      }
+    }
+
+    private static class getSystemProperties_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<getSystemProperties_result> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, getSystemProperties_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 0: // SUCCESS
+              if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+                {
+                  org.apache.thrift.protocol.TMap _map84 = iprot.readMapBegin();
+                  struct.success = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map84.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _key85;
+                  @org.apache.thrift.annotation.Nullable java.lang.String _val86;
+                  for (int _i87 = 0; _i87 < _map84.size; ++_i87)
+                  {
+                    _key85 = iprot.readString();
+                    _val86 = iprot.readString();
+                    struct.success.put(_key85, _val86);
+                  }
+                  iprot.readMapEnd();
+                }
+                struct.setSuccessIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+
+        // check for required fields of primitive type, which can't be checked in the validate method
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, getSystemProperties_result struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.success != null) {
+          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+          {
+            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.success.size()));
+            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter88 : struct.success.entrySet())
+            {
+              oprot.writeString(_iter88.getKey());
+              oprot.writeString(_iter88.getValue());
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class getSystemProperties_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getSystemProperties_resultTupleScheme getScheme() {
+        return new getSystemProperties_resultTupleScheme();
+      }
+    }
+
+    private static class getSystemProperties_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<getSystemProperties_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, getSystemProperties_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet optionals = new java.util.BitSet();
+        if (struct.isSetSuccess()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetSuccess()) {
+          {
+            oprot.writeI32(struct.success.size());
+            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter89 : struct.success.entrySet())
+            {
+              oprot.writeString(_iter89.getKey());
+              oprot.writeString(_iter89.getValue());
+            }
+          }
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, getSystemProperties_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)) {
+          {
+            org.apache.thrift.protocol.TMap _map90 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); 
+            struct.success = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map90.size);
+            @org.apache.thrift.annotation.Nullable java.lang.String _key91;
+            @org.apache.thrift.annotation.Nullable java.lang.String _val92;
+            for (int _i93 = 0; _i93 < _map90.size; ++_i93)
+            {
+              _key91 = iprot.readString();
+              _val92 = iprot.readString();
+              struct.success.put(_key91, _val92);
+            }
+          }
+          struct.setSuccessIsSet(true);
+        }
+      }
+    }
+
+    private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
+      return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
+    }
+  }
+
+  public static class getVersionedSystemProperties_args implements org.apache.thrift.TBase<getVersionedSystemProperties_args, getVersionedSystemProperties_args._Fields>, java.io.Serializable, Cloneable, Comparable<getVersionedSystemProperties_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getVersionedSystemProperties_args");
+
+    private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+    private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getVersionedSystemProperties_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getVersionedSystemProperties_argsTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // 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 {
+      TINFO((short)1, "tinfo"),
+      CREDENTIALS((short)2, "credentials");
+
+      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: // TINFO
+            return TINFO;
+          case 2: // CREDENTIALS
+            return CREDENTIALS;
+          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.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+      tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getVersionedSystemProperties_args.class, metaDataMap);
+    }
+
+    public getVersionedSystemProperties_args() {
+    }
+
+    public getVersionedSystemProperties_args(
+      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials)
+    {
+      this();
+      this.tinfo = tinfo;
+      this.credentials = credentials;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public getVersionedSystemProperties_args(getVersionedSystemProperties_args other) {
+      if (other.isSetTinfo()) {
+        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+      }
+      if (other.isSetCredentials()) {
+        this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
+      }
+    }
+
+    public getVersionedSystemProperties_args deepCopy() {
+      return new getVersionedSystemProperties_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.tinfo = null;
+      this.credentials = null;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+      return this.tinfo;
+    }
+
+    public getVersionedSystemProperties_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+      this.tinfo = tinfo;
+      return this;
+    }
+
+    public void unsetTinfo() {
+      this.tinfo = null;
+    }
+
+    /** Returns true if field tinfo is set (has been assigned a value) and false otherwise */
+    public boolean isSetTinfo() {
+      return this.tinfo != null;
+    }
+
+    public void setTinfoIsSet(boolean value) {
+      if (!value) {
+        this.tinfo = null;
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.securityImpl.thrift.TCredentials getCredentials() {
+      return this.credentials;
+    }
+
+    public getVersionedSystemProperties_args setCredentials(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) {
+      this.credentials = credentials;
+      return this;
+    }
+
+    public void unsetCredentials() {
+      this.credentials = null;
+    }
+
+    /** Returns true if field credentials is set (has been assigned a value) and false otherwise */
+    public boolean isSetCredentials() {
+      return this.credentials != null;
+    }
+
+    public void setCredentialsIsSet(boolean value) {
+      if (!value) {
+        this.credentials = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+      switch (field) {
+      case TINFO:
+        if (value == null) {
+          unsetTinfo();
+        } else {
+          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+        }
+        break;
+
+      case CREDENTIALS:
+        if (value == null) {
+          unsetCredentials();
+        } else {
+          setCredentials((org.apache.accumulo.core.securityImpl.thrift.TCredentials)value);
+        }
+        break;
+
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case TINFO:
+        return getTinfo();
+
+      case CREDENTIALS:
+        return getCredentials();
+
+      }
+      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 TINFO:
+        return isSetTinfo();
+      case CREDENTIALS:
+        return isSetCredentials();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that instanceof getVersionedSystemProperties_args)
+        return this.equals((getVersionedSystemProperties_args)that);
+      return false;
+    }
+
+    public boolean equals(getVersionedSystemProperties_args that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return true;
+
+      boolean this_present_tinfo = true && this.isSetTinfo();
+      boolean that_present_tinfo = true && that.isSetTinfo();
+      if (this_present_tinfo || that_present_tinfo) {
+        if (!(this_present_tinfo && that_present_tinfo))
+          return false;
+        if (!this.tinfo.equals(that.tinfo))
+          return false;
+      }
+
+      boolean this_present_credentials = true && this.isSetCredentials();
+      boolean that_present_credentials = true && that.isSetCredentials();
+      if (this_present_credentials || that_present_credentials) {
+        if (!(this_present_credentials && that_present_credentials))
+          return false;
+        if (!this.credentials.equals(that.credentials))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      hashCode = hashCode * 8191 + ((isSetTinfo()) ? 131071 : 524287);
+      if (isSetTinfo())
+        hashCode = hashCode * 8191 + tinfo.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetCredentials()) ? 131071 : 524287);
+      if (isSetCredentials())
+        hashCode = hashCode * 8191 + credentials.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(getVersionedSystemProperties_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = java.lang.Boolean.compare(isSetTinfo(), other.isSetTinfo());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetTinfo()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tinfo, other.tinfo);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.compare(isSetCredentials(), other.isSetCredentials());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetCredentials()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.credentials, other.credentials);
+        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("getVersionedSystemProperties_args(");
+      boolean first = true;
+
+      sb.append("tinfo:");
+      if (this.tinfo == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.tinfo);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("credentials:");
+      if (this.credentials == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.credentials);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+      if (tinfo != null) {
+        tinfo.validate();
+      }
+      if (credentials != null) {
+        credentials.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 getVersionedSystemProperties_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getVersionedSystemProperties_argsStandardScheme getScheme() {
+        return new getVersionedSystemProperties_argsStandardScheme();
+      }
+    }
+
+    private static class getVersionedSystemProperties_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<getVersionedSystemProperties_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, getVersionedSystemProperties_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: // TINFO
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+                struct.tinfo.read(iprot);
+                struct.setTinfoIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 2: // CREDENTIALS
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
+                struct.credentials.read(iprot);
+                struct.setCredentialsIsSet(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, getVersionedSystemProperties_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.tinfo != null) {
+          oprot.writeFieldBegin(TINFO_FIELD_DESC);
+          struct.tinfo.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.credentials != null) {
+          oprot.writeFieldBegin(CREDENTIALS_FIELD_DESC);
+          struct.credentials.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class getVersionedSystemProperties_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getVersionedSystemProperties_argsTupleScheme getScheme() {
+        return new getVersionedSystemProperties_argsTupleScheme();
+      }
+    }
+
+    private static class getVersionedSystemProperties_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<getVersionedSystemProperties_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, getVersionedSystemProperties_args 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.isSetTinfo()) {
+          optionals.set(0);
+        }
+        if (struct.isSetCredentials()) {
+          optionals.set(1);
+        }
+        oprot.writeBitSet(optionals, 2);
+        if (struct.isSetTinfo()) {
+          struct.tinfo.write(oprot);
+        }
+        if (struct.isSetCredentials()) {
+          struct.credentials.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, getVersionedSystemProperties_args 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.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+          struct.tinfo.read(iprot);
+          struct.setTinfoIsSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
+          struct.credentials.read(iprot);
+          struct.setCredentialsIsSet(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 getVersionedSystemProperties_result implements org.apache.thrift.TBase<getVersionedSystemProperties_result, getVersionedSystemProperties_result._Fields>, java.io.Serializable, Cloneable, Comparable<getVersionedSystemProperties_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getVersionedSystemProperties_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.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getVersionedSystemProperties_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getVersionedSystemProperties_resultTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable TVersionedProperties success; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      SUCCESS((short)0, "success");
+
+      private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
+
+      static {
+        for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      @org.apache.thrift.annotation.Nullable
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 0: // SUCCESS
+            return SUCCESS;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      @org.apache.thrift.annotation.Nullable
+      public static _Fields findByName(java.lang.String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final java.lang.String _fieldName;
+
+      _Fields(short thriftId, java.lang.String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public java.lang.String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TVersionedProperties.class)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getVersionedSystemProperties_result.class, metaDataMap);
+    }
+
+    public getVersionedSystemProperties_result() {
+    }
+
+    public getVersionedSystemProperties_result(
+      TVersionedProperties success)
+    {
+      this();
+      this.success = success;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public getVersionedSystemProperties_result(getVersionedSystemProperties_result other) {
+      if (other.isSetSuccess()) {
+        this.success = new TVersionedProperties(other.success);
+      }
+    }
+
+    public getVersionedSystemProperties_result deepCopy() {
+      return new getVersionedSystemProperties_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.success = null;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public TVersionedProperties getSuccess() {
+      return this.success;
+    }
+
+    public getVersionedSystemProperties_result setSuccess(@org.apache.thrift.annotation.Nullable TVersionedProperties success) {
+      this.success = success;
+      return this;
+    }
+
+    public void unsetSuccess() {
+      this.success = null;
+    }
+
+    /** Returns true if field success is set (has been assigned a value) and false otherwise */
+    public boolean isSetSuccess() {
+      return this.success != null;
+    }
+
+    public void setSuccessIsSet(boolean value) {
+      if (!value) {
+        this.success = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+      switch (field) {
+      case SUCCESS:
+        if (value == null) {
+          unsetSuccess();
+        } else {
+          setSuccess((TVersionedProperties)value);
+        }
+        break;
+
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return getSuccess();
+
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case SUCCESS:
+        return isSetSuccess();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that instanceof getVersionedSystemProperties_result)
+        return this.equals((getVersionedSystemProperties_result)that);
+      return false;
+    }
+
+    public boolean equals(getVersionedSystemProperties_result that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return true;
+
+      boolean this_present_success = true && this.isSetSuccess();
+      boolean that_present_success = true && that.isSetSuccess();
+      if (this_present_success || that_present_success) {
+        if (!(this_present_success && that_present_success))
+          return false;
+        if (!this.success.equals(that.success))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      hashCode = hashCode * 8191 + ((isSetSuccess()) ? 131071 : 524287);
+      if (isSetSuccess())
+        hashCode = hashCode * 8191 + success.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(getVersionedSystemProperties_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = java.lang.Boolean.compare(isSetSuccess(), other.isSetSuccess());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetSuccess()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      scheme(iprot).read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      scheme(oprot).write(oprot, this);
+      }
+
+    @Override
+    public java.lang.String toString() {
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("getVersionedSystemProperties_result(");
+      boolean first = true;
+
+      sb.append("success:");
+      if (this.success == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.success);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+      if (success != null) {
+        success.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 getVersionedSystemProperties_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getVersionedSystemProperties_resultStandardScheme getScheme() {
+        return new getVersionedSystemProperties_resultStandardScheme();
+      }
+    }
+
+    private static class getVersionedSystemProperties_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<getVersionedSystemProperties_result> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, getVersionedSystemProperties_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 0: // SUCCESS
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.success = new TVersionedProperties();
+                struct.success.read(iprot);
+                struct.setSuccessIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+
+        // check for required fields of primitive type, which can't be checked in the validate method
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, getVersionedSystemProperties_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();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class getVersionedSystemProperties_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getVersionedSystemProperties_resultTupleScheme getScheme() {
+        return new getVersionedSystemProperties_resultTupleScheme();
+      }
+    }
+
+    private static class getVersionedSystemProperties_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<getVersionedSystemProperties_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, getVersionedSystemProperties_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet optionals = new java.util.BitSet();
+        if (struct.isSetSuccess()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetSuccess()) {
+          struct.success.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, getVersionedSystemProperties_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          struct.success = new TVersionedProperties();
+          struct.success.read(iprot);
+          struct.setSuccessIsSet(true);
+        }
+      }
+    }
+
+    private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
+      return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
+    }
+  }
+
+  public static class getTableConfiguration_args implements org.apache.thrift.TBase<getTableConfiguration_args, getTableConfiguration_args._Fields>, java.io.Serializable, Cloneable, Comparable<getTableConfiguration_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getTableConfiguration_args");
+
+    private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+    private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)3);
+    private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)2);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getTableConfiguration_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getTableConfiguration_argsTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
+    public @org.apache.thrift.annotation.Nullable java.lang.String 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 {
+      TINFO((short)1, "tinfo"),
+      CREDENTIALS((short)3, "credentials"),
+      TABLE_NAME((short)2, "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: // TINFO
+            return TINFO;
+          case 3: // CREDENTIALS
+            return CREDENTIALS;
+          case 2: // 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.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+      tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
+      tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getTableConfiguration_args.class, metaDataMap);
+    }
+
+    public getTableConfiguration_args() {
+    }
+
+    public getTableConfiguration_args(
+      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
+      java.lang.String tableName)
+    {
+      this();
+      this.tinfo = tinfo;
+      this.credentials = credentials;
+      this.tableName = tableName;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public getTableConfiguration_args(getTableConfiguration_args other) {
+      if (other.isSetTinfo()) {
+        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+      }
+      if (other.isSetCredentials()) {
+        this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
+      }
+      if (other.isSetTableName()) {
+        this.tableName = other.tableName;
+      }
+    }
+
+    public getTableConfiguration_args deepCopy() {
+      return new getTableConfiguration_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.tinfo = null;
+      this.credentials = null;
+      this.tableName = null;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+      return this.tinfo;
+    }
+
+    public getTableConfiguration_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+      this.tinfo = tinfo;
+      return this;
+    }
+
+    public void unsetTinfo() {
+      this.tinfo = null;
+    }
+
+    /** Returns true if field tinfo is set (has been assigned a value) and false otherwise */
+    public boolean isSetTinfo() {
+      return this.tinfo != null;
+    }
+
+    public void setTinfoIsSet(boolean value) {
+      if (!value) {
+        this.tinfo = null;
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.securityImpl.thrift.TCredentials getCredentials() {
+      return this.credentials;
+    }
+
+    public getTableConfiguration_args setCredentials(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) {
+      this.credentials = credentials;
+      return this;
+    }
+
+    public void unsetCredentials() {
+      this.credentials = null;
+    }
+
+    /** Returns true if field credentials is set (has been assigned a value) and false otherwise */
+    public boolean isSetCredentials() {
+      return this.credentials != null;
+    }
+
+    public void setCredentialsIsSet(boolean value) {
+      if (!value) {
+        this.credentials = null;
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.lang.String getTableName() {
+      return this.tableName;
+    }
+
+    public getTableConfiguration_args setTableName(@org.apache.thrift.annotation.Nullable java.lang.String 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 TINFO:
+        if (value == null) {
+          unsetTinfo();
+        } else {
+          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+        }
+        break;
+
+      case CREDENTIALS:
+        if (value == null) {
+          unsetCredentials();
+        } else {
+          setCredentials((org.apache.accumulo.core.securityImpl.thrift.TCredentials)value);
+        }
+        break;
+
+      case TABLE_NAME:
+        if (value == null) {
+          unsetTableName();
+        } else {
+          setTableName((java.lang.String)value);
+        }
+        break;
+
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case TINFO:
+        return getTinfo();
+
+      case CREDENTIALS:
+        return getCredentials();
+
+      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 TINFO:
+        return isSetTinfo();
+      case CREDENTIALS:
+        return isSetCredentials();
+      case TABLE_NAME:
+        return isSetTableName();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that instanceof getTableConfiguration_args)
+        return this.equals((getTableConfiguration_args)that);
+      return false;
+    }
+
+    public boolean equals(getTableConfiguration_args that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return true;
+
+      boolean this_present_tinfo = true && this.isSetTinfo();
+      boolean that_present_tinfo = true && that.isSetTinfo();
+      if (this_present_tinfo || that_present_tinfo) {
+        if (!(this_present_tinfo && that_present_tinfo))
+          return false;
+        if (!this.tinfo.equals(that.tinfo))
+          return false;
+      }
+
+      boolean this_present_credentials = true && this.isSetCredentials();
+      boolean that_present_credentials = true && that.isSetCredentials();
+      if (this_present_credentials || that_present_credentials) {
+        if (!(this_present_credentials && that_present_credentials))
+          return false;
+        if (!this.credentials.equals(that.credentials))
+          return false;
+      }
+
+      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 + ((isSetTinfo()) ? 131071 : 524287);
+      if (isSetTinfo())
+        hashCode = hashCode * 8191 + tinfo.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetCredentials()) ? 131071 : 524287);
+      if (isSetCredentials())
+        hashCode = hashCode * 8191 + credentials.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetTableName()) ? 131071 : 524287);
+      if (isSetTableName())
+        hashCode = hashCode * 8191 + tableName.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(getTableConfiguration_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = java.lang.Boolean.compare(isSetTinfo(), other.isSetTinfo());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetTinfo()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tinfo, other.tinfo);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.compare(isSetCredentials(), other.isSetCredentials());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetCredentials()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.credentials, other.credentials);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.compare(isSetTableName(), 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("getTableConfiguration_args(");
+      boolean first = true;
+
+      sb.append("tinfo:");
+      if (this.tinfo == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.tinfo);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("credentials:");
+      if (this.credentials == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.credentials);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      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
+      // check for sub-struct validity
+      if (tinfo != null) {
+        tinfo.validate();
+      }
+      if (credentials != null) {
+        credentials.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 getTableConfiguration_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getTableConfiguration_argsStandardScheme getScheme() {
+        return new getTableConfiguration_argsStandardScheme();
+      }
+    }
+
+    private static class getTableConfiguration_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<getTableConfiguration_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, getTableConfiguration_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: // TINFO
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+                struct.tinfo.read(iprot);
+                struct.setTinfoIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 3: // CREDENTIALS
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
+                struct.credentials.read(iprot);
+                struct.setCredentialsIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 2: // TABLE_NAME
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.tableName = iprot.readString();
+                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, getTableConfiguration_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.tinfo != null) {
+          oprot.writeFieldBegin(TINFO_FIELD_DESC);
+          struct.tinfo.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.tableName != null) {
+          oprot.writeFieldBegin(TABLE_NAME_FIELD_DESC);
+          oprot.writeString(struct.tableName);
+          oprot.writeFieldEnd();
+        }
+        if (struct.credentials != null) {
+          oprot.writeFieldBegin(CREDENTIALS_FIELD_DESC);
+          struct.credentials.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class getTableConfiguration_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getTableConfiguration_argsTupleScheme getScheme() {
+        return new getTableConfiguration_argsTupleScheme();
+      }
+    }
+
+    private static class getTableConfiguration_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<getTableConfiguration_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, getTableConfiguration_args 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.isSetTinfo()) {
+          optionals.set(0);
+        }
+        if (struct.isSetCredentials()) {
+          optionals.set(1);
+        }
+        if (struct.isSetTableName()) {
+          optionals.set(2);
+        }
+        oprot.writeBitSet(optionals, 3);
+        if (struct.isSetTinfo()) {
+          struct.tinfo.write(oprot);
+        }
+        if (struct.isSetCredentials()) {
+          struct.credentials.write(oprot);
+        }
+        if (struct.isSetTableName()) {
+          oprot.writeString(struct.tableName);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, getTableConfiguration_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet incoming = iprot.readBitSet(3);
+        if (incoming.get(0)) {
+          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+          struct.tinfo.read(iprot);
+          struct.setTinfoIsSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
+          struct.credentials.read(iprot);
+          struct.setCredentialsIsSet(true);
+        }
+        if (incoming.get(2)) {
+          struct.tableName = iprot.readString();
+          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 getTableConfiguration_result implements org.apache.thrift.TBase<getTableConfiguration_result, getTableConfiguration_result._Fields>, java.io.Serializable, Cloneable, Comparable<getTableConfiguration_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getTableConfiguration_result");
+
+    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.MAP, (short)0);
+    private static final org.apache.thrift.protocol.TField TOPE_FIELD_DESC = new org.apache.thrift.protocol.TField("tope", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getTableConfiguration_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getTableConfiguration_resultTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,java.lang.String> success; // required
+    public @org.apache.thrift.annotation.Nullable ThriftTableOperationException tope; // 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"),
+      TOPE((short)1, "tope");
+
+      private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
+
+      static {
+        for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      @org.apache.thrift.annotation.Nullable
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 0: // SUCCESS
+            return SUCCESS;
+          case 1: // TOPE
+            return TOPE;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      @org.apache.thrift.annotation.Nullable
+      public static _Fields findByName(java.lang.String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final java.lang.String _fieldName;
+
+      _Fields(short thriftId, java.lang.String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public java.lang.String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+              new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+              new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+      tmpMap.put(_Fields.TOPE, new org.apache.thrift.meta_data.FieldMetaData("tope", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ThriftTableOperationException.class)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getTableConfiguration_result.class, metaDataMap);
+    }
+
+    public getTableConfiguration_result() {
+    }
+
+    public getTableConfiguration_result(
+      java.util.Map<java.lang.String,java.lang.String> success,
+      ThriftTableOperationException tope)
+    {
+      this();
+      this.success = success;
+      this.tope = tope;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public getTableConfiguration_result(getTableConfiguration_result other) {
+      if (other.isSetSuccess()) {
+        java.util.Map<java.lang.String,java.lang.String> __this__success = new java.util.HashMap<java.lang.String,java.lang.String>(other.success);
+        this.success = __this__success;
+      }
+      if (other.isSetTope()) {
+        this.tope = new ThriftTableOperationException(other.tope);
+      }
+    }
+
+    public getTableConfiguration_result deepCopy() {
+      return new getTableConfiguration_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.success = null;
+      this.tope = null;
+    }
+
+    public int getSuccessSize() {
+      return (this.success == null) ? 0 : this.success.size();
+    }
+
+    public void putToSuccess(java.lang.String key, java.lang.String val) {
+      if (this.success == null) {
+        this.success = new java.util.HashMap<java.lang.String,java.lang.String>();
+      }
+      this.success.put(key, val);
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.util.Map<java.lang.String,java.lang.String> getSuccess() {
+      return this.success;
+    }
+
+    public getTableConfiguration_result setSuccess(@org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,java.lang.String> success) {
+      this.success = success;
+      return this;
+    }
+
+    public void unsetSuccess() {
+      this.success = null;
+    }
+
+    /** Returns true if field success is set (has been assigned a value) and false otherwise */
+    public boolean isSetSuccess() {
+      return this.success != null;
+    }
+
+    public void setSuccessIsSet(boolean value) {
+      if (!value) {
+        this.success = null;
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public ThriftTableOperationException getTope() {
+      return this.tope;
+    }
+
+    public getTableConfiguration_result setTope(@org.apache.thrift.annotation.Nullable ThriftTableOperationException tope) {
+      this.tope = tope;
+      return this;
+    }
+
+    public void unsetTope() {
+      this.tope = null;
+    }
+
+    /** Returns true if field tope is set (has been assigned a value) and false otherwise */
+    public boolean isSetTope() {
+      return this.tope != null;
+    }
+
+    public void setTopeIsSet(boolean value) {
+      if (!value) {
+        this.tope = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+      switch (field) {
+      case SUCCESS:
+        if (value == null) {
+          unsetSuccess();
+        } else {
+          setSuccess((java.util.Map<java.lang.String,java.lang.String>)value);
+        }
+        break;
+
+      case TOPE:
+        if (value == null) {
+          unsetTope();
+        } else {
+          setTope((ThriftTableOperationException)value);
+        }
+        break;
+
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return getSuccess();
+
+      case TOPE:
+        return getTope();
+
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case SUCCESS:
+        return isSetSuccess();
+      case TOPE:
+        return isSetTope();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that instanceof getTableConfiguration_result)
+        return this.equals((getTableConfiguration_result)that);
+      return false;
+    }
+
+    public boolean equals(getTableConfiguration_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_tope = true && this.isSetTope();
+      boolean that_present_tope = true && that.isSetTope();
+      if (this_present_tope || that_present_tope) {
+        if (!(this_present_tope && that_present_tope))
+          return false;
+        if (!this.tope.equals(that.tope))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      hashCode = hashCode * 8191 + ((isSetSuccess()) ? 131071 : 524287);
+      if (isSetSuccess())
+        hashCode = hashCode * 8191 + success.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetTope()) ? 131071 : 524287);
+      if (isSetTope())
+        hashCode = hashCode * 8191 + tope.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(getTableConfiguration_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = java.lang.Boolean.compare(isSetSuccess(), 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.compare(isSetTope(), other.isSetTope());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetTope()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tope, other.tope);
+        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("getTableConfiguration_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("tope:");
+      if (this.tope == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.tope);
+      }
+      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 getTableConfiguration_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getTableConfiguration_resultStandardScheme getScheme() {
+        return new getTableConfiguration_resultStandardScheme();
+      }
+    }
+
+    private static class getTableConfiguration_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<getTableConfiguration_result> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, getTableConfiguration_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 0: // SUCCESS
+              if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+                {
+                  org.apache.thrift.protocol.TMap _map94 = iprot.readMapBegin();
+                  struct.success = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map94.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _key95;
+                  @org.apache.thrift.annotation.Nullable java.lang.String _val96;
+                  for (int _i97 = 0; _i97 < _map94.size; ++_i97)
+                  {
+                    _key95 = iprot.readString();
+                    _val96 = iprot.readString();
+                    struct.success.put(_key95, _val96);
+                  }
+                  iprot.readMapEnd();
+                }
+                struct.setSuccessIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 1: // TOPE
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.tope = new ThriftTableOperationException();
+                struct.tope.read(iprot);
+                struct.setTopeIsSet(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, getTableConfiguration_result struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.success != null) {
+          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+          {
+            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.success.size()));
+            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter98 : struct.success.entrySet())
+            {
+              oprot.writeString(_iter98.getKey());
+              oprot.writeString(_iter98.getValue());
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+        if (struct.tope != null) {
+          oprot.writeFieldBegin(TOPE_FIELD_DESC);
+          struct.tope.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class getTableConfiguration_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getTableConfiguration_resultTupleScheme getScheme() {
+        return new getTableConfiguration_resultTupleScheme();
+      }
+    }
+
+    private static class getTableConfiguration_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<getTableConfiguration_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, getTableConfiguration_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.isSetTope()) {
+          optionals.set(1);
+        }
+        oprot.writeBitSet(optionals, 2);
+        if (struct.isSetSuccess()) {
+          {
+            oprot.writeI32(struct.success.size());
+            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter99 : struct.success.entrySet())
+            {
+              oprot.writeString(_iter99.getKey());
+              oprot.writeString(_iter99.getValue());
+            }
+          }
+        }
+        if (struct.isSetTope()) {
+          struct.tope.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, getTableConfiguration_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)) {
+          {
+            org.apache.thrift.protocol.TMap _map100 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); 
+            struct.success = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map100.size);
+            @org.apache.thrift.annotation.Nullable java.lang.String _key101;
+            @org.apache.thrift.annotation.Nullable java.lang.String _val102;
+            for (int _i103 = 0; _i103 < _map100.size; ++_i103)
+            {
+              _key101 = iprot.readString();
+              _val102 = iprot.readString();
+              struct.success.put(_key101, _val102);
+            }
+          }
+          struct.setSuccessIsSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.tope = new ThriftTableOperationException();
+          struct.tope.read(iprot);
+          struct.setTopeIsSet(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 getTableProperties_args implements org.apache.thrift.TBase<getTableProperties_args, getTableProperties_args._Fields>, java.io.Serializable, Cloneable, Comparable<getTableProperties_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getTableProperties_args");
+
+    private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+    private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)3);
+    private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)2);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getTableProperties_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getTableProperties_argsTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
+    public @org.apache.thrift.annotation.Nullable java.lang.String 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 {
+      TINFO((short)1, "tinfo"),
+      CREDENTIALS((short)3, "credentials"),
+      TABLE_NAME((short)2, "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: // TINFO
+            return TINFO;
+          case 3: // CREDENTIALS
+            return CREDENTIALS;
+          case 2: // 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.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+      tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
+      tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getTableProperties_args.class, metaDataMap);
+    }
+
+    public getTableProperties_args() {
+    }
+
+    public getTableProperties_args(
+      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
+      java.lang.String tableName)
+    {
+      this();
+      this.tinfo = tinfo;
+      this.credentials = credentials;
+      this.tableName = tableName;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public getTableProperties_args(getTableProperties_args other) {
+      if (other.isSetTinfo()) {
+        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+      }
+      if (other.isSetCredentials()) {
+        this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
+      }
+      if (other.isSetTableName()) {
+        this.tableName = other.tableName;
+      }
+    }
+
+    public getTableProperties_args deepCopy() {
+      return new getTableProperties_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.tinfo = null;
+      this.credentials = null;
+      this.tableName = null;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+      return this.tinfo;
+    }
+
+    public getTableProperties_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+      this.tinfo = tinfo;
+      return this;
+    }
+
+    public void unsetTinfo() {
+      this.tinfo = null;
+    }
+
+    /** Returns true if field tinfo is set (has been assigned a value) and false otherwise */
+    public boolean isSetTinfo() {
+      return this.tinfo != null;
+    }
+
+    public void setTinfoIsSet(boolean value) {
+      if (!value) {
+        this.tinfo = null;
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.securityImpl.thrift.TCredentials getCredentials() {
+      return this.credentials;
+    }
+
+    public getTableProperties_args setCredentials(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) {
+      this.credentials = credentials;
+      return this;
+    }
+
+    public void unsetCredentials() {
+      this.credentials = null;
+    }
+
+    /** Returns true if field credentials is set (has been assigned a value) and false otherwise */
+    public boolean isSetCredentials() {
+      return this.credentials != null;
+    }
+
+    public void setCredentialsIsSet(boolean value) {
+      if (!value) {
+        this.credentials = null;
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.lang.String getTableName() {
+      return this.tableName;
+    }
+
+    public getTableProperties_args setTableName(@org.apache.thrift.annotation.Nullable java.lang.String 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 TINFO:
+        if (value == null) {
+          unsetTinfo();
+        } else {
+          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+        }
+        break;
+
+      case CREDENTIALS:
+        if (value == null) {
+          unsetCredentials();
+        } else {
+          setCredentials((org.apache.accumulo.core.securityImpl.thrift.TCredentials)value);
+        }
+        break;
+
+      case TABLE_NAME:
+        if (value == null) {
+          unsetTableName();
+        } else {
+          setTableName((java.lang.String)value);
+        }
+        break;
+
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case TINFO:
+        return getTinfo();
+
+      case CREDENTIALS:
+        return getCredentials();
+
+      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 TINFO:
+        return isSetTinfo();
+      case CREDENTIALS:
+        return isSetCredentials();
+      case TABLE_NAME:
+        return isSetTableName();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that instanceof getTableProperties_args)
+        return this.equals((getTableProperties_args)that);
+      return false;
+    }
+
+    public boolean equals(getTableProperties_args that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return true;
+
+      boolean this_present_tinfo = true && this.isSetTinfo();
+      boolean that_present_tinfo = true && that.isSetTinfo();
+      if (this_present_tinfo || that_present_tinfo) {
+        if (!(this_present_tinfo && that_present_tinfo))
+          return false;
+        if (!this.tinfo.equals(that.tinfo))
+          return false;
+      }
+
+      boolean this_present_credentials = true && this.isSetCredentials();
+      boolean that_present_credentials = true && that.isSetCredentials();
+      if (this_present_credentials || that_present_credentials) {
+        if (!(this_present_credentials && that_present_credentials))
+          return false;
+        if (!this.credentials.equals(that.credentials))
+          return false;
+      }
+
+      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 + ((isSetTinfo()) ? 131071 : 524287);
+      if (isSetTinfo())
+        hashCode = hashCode * 8191 + tinfo.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetCredentials()) ? 131071 : 524287);
+      if (isSetCredentials())
+        hashCode = hashCode * 8191 + credentials.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetTableName()) ? 131071 : 524287);
+      if (isSetTableName())
+        hashCode = hashCode * 8191 + tableName.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(getTableProperties_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = java.lang.Boolean.compare(isSetTinfo(), other.isSetTinfo());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetTinfo()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tinfo, other.tinfo);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.compare(isSetCredentials(), other.isSetCredentials());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetCredentials()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.credentials, other.credentials);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.compare(isSetTableName(), 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("getTableProperties_args(");
+      boolean first = true;
+
+      sb.append("tinfo:");
+      if (this.tinfo == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.tinfo);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("credentials:");
+      if (this.credentials == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.credentials);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      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
+      // check for sub-struct validity
+      if (tinfo != null) {
+        tinfo.validate();
+      }
+      if (credentials != null) {
+        credentials.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 getTableProperties_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getTableProperties_argsStandardScheme getScheme() {
+        return new getTableProperties_argsStandardScheme();
+      }
+    }
+
+    private static class getTableProperties_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<getTableProperties_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, getTableProperties_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: // TINFO
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+                struct.tinfo.read(iprot);
+                struct.setTinfoIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 3: // CREDENTIALS
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
+                struct.credentials.read(iprot);
+                struct.setCredentialsIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 2: // TABLE_NAME
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.tableName = iprot.readString();
+                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, getTableProperties_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.tinfo != null) {
+          oprot.writeFieldBegin(TINFO_FIELD_DESC);
+          struct.tinfo.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.tableName != null) {
+          oprot.writeFieldBegin(TABLE_NAME_FIELD_DESC);
+          oprot.writeString(struct.tableName);
+          oprot.writeFieldEnd();
+        }
+        if (struct.credentials != null) {
+          oprot.writeFieldBegin(CREDENTIALS_FIELD_DESC);
+          struct.credentials.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class getTableProperties_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getTableProperties_argsTupleScheme getScheme() {
+        return new getTableProperties_argsTupleScheme();
+      }
+    }
+
+    private static class getTableProperties_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<getTableProperties_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, getTableProperties_args 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.isSetTinfo()) {
+          optionals.set(0);
+        }
+        if (struct.isSetCredentials()) {
+          optionals.set(1);
+        }
+        if (struct.isSetTableName()) {
+          optionals.set(2);
+        }
+        oprot.writeBitSet(optionals, 3);
+        if (struct.isSetTinfo()) {
+          struct.tinfo.write(oprot);
+        }
+        if (struct.isSetCredentials()) {
+          struct.credentials.write(oprot);
+        }
+        if (struct.isSetTableName()) {
+          oprot.writeString(struct.tableName);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, getTableProperties_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet incoming = iprot.readBitSet(3);
+        if (incoming.get(0)) {
+          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+          struct.tinfo.read(iprot);
+          struct.setTinfoIsSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
+          struct.credentials.read(iprot);
+          struct.setCredentialsIsSet(true);
+        }
+        if (incoming.get(2)) {
+          struct.tableName = iprot.readString();
+          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 getTableProperties_result implements org.apache.thrift.TBase<getTableProperties_result, getTableProperties_result._Fields>, java.io.Serializable, Cloneable, Comparable<getTableProperties_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getTableProperties_result");
+
+    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.MAP, (short)0);
+    private static final org.apache.thrift.protocol.TField TOPE_FIELD_DESC = new org.apache.thrift.protocol.TField("tope", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getTableProperties_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getTableProperties_resultTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,java.lang.String> success; // required
+    public @org.apache.thrift.annotation.Nullable ThriftTableOperationException tope; // 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"),
+      TOPE((short)1, "tope");
+
+      private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
+
+      static {
+        for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      @org.apache.thrift.annotation.Nullable
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 0: // SUCCESS
+            return SUCCESS;
+          case 1: // TOPE
+            return TOPE;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      @org.apache.thrift.annotation.Nullable
+      public static _Fields findByName(java.lang.String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final java.lang.String _fieldName;
+
+      _Fields(short thriftId, java.lang.String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public java.lang.String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+              new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+              new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+      tmpMap.put(_Fields.TOPE, new org.apache.thrift.meta_data.FieldMetaData("tope", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ThriftTableOperationException.class)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getTableProperties_result.class, metaDataMap);
+    }
+
+    public getTableProperties_result() {
+    }
+
+    public getTableProperties_result(
+      java.util.Map<java.lang.String,java.lang.String> success,
+      ThriftTableOperationException tope)
+    {
+      this();
+      this.success = success;
+      this.tope = tope;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public getTableProperties_result(getTableProperties_result other) {
+      if (other.isSetSuccess()) {
+        java.util.Map<java.lang.String,java.lang.String> __this__success = new java.util.HashMap<java.lang.String,java.lang.String>(other.success);
+        this.success = __this__success;
+      }
+      if (other.isSetTope()) {
+        this.tope = new ThriftTableOperationException(other.tope);
+      }
+    }
+
+    public getTableProperties_result deepCopy() {
+      return new getTableProperties_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.success = null;
+      this.tope = null;
+    }
+
+    public int getSuccessSize() {
+      return (this.success == null) ? 0 : this.success.size();
+    }
+
+    public void putToSuccess(java.lang.String key, java.lang.String val) {
+      if (this.success == null) {
+        this.success = new java.util.HashMap<java.lang.String,java.lang.String>();
+      }
+      this.success.put(key, val);
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.util.Map<java.lang.String,java.lang.String> getSuccess() {
+      return this.success;
+    }
+
+    public getTableProperties_result setSuccess(@org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,java.lang.String> success) {
+      this.success = success;
+      return this;
+    }
+
+    public void unsetSuccess() {
+      this.success = null;
+    }
+
+    /** Returns true if field success is set (has been assigned a value) and false otherwise */
+    public boolean isSetSuccess() {
+      return this.success != null;
+    }
+
+    public void setSuccessIsSet(boolean value) {
+      if (!value) {
+        this.success = null;
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public ThriftTableOperationException getTope() {
+      return this.tope;
+    }
+
+    public getTableProperties_result setTope(@org.apache.thrift.annotation.Nullable ThriftTableOperationException tope) {
+      this.tope = tope;
+      return this;
+    }
+
+    public void unsetTope() {
+      this.tope = null;
+    }
+
+    /** Returns true if field tope is set (has been assigned a value) and false otherwise */
+    public boolean isSetTope() {
+      return this.tope != null;
+    }
+
+    public void setTopeIsSet(boolean value) {
+      if (!value) {
+        this.tope = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+      switch (field) {
+      case SUCCESS:
+        if (value == null) {
+          unsetSuccess();
+        } else {
+          setSuccess((java.util.Map<java.lang.String,java.lang.String>)value);
+        }
+        break;
+
+      case TOPE:
+        if (value == null) {
+          unsetTope();
+        } else {
+          setTope((ThriftTableOperationException)value);
+        }
+        break;
+
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return getSuccess();
+
+      case TOPE:
+        return getTope();
+
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case SUCCESS:
+        return isSetSuccess();
+      case TOPE:
+        return isSetTope();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that instanceof getTableProperties_result)
+        return this.equals((getTableProperties_result)that);
+      return false;
+    }
+
+    public boolean equals(getTableProperties_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_tope = true && this.isSetTope();
+      boolean that_present_tope = true && that.isSetTope();
+      if (this_present_tope || that_present_tope) {
+        if (!(this_present_tope && that_present_tope))
+          return false;
+        if (!this.tope.equals(that.tope))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      hashCode = hashCode * 8191 + ((isSetSuccess()) ? 131071 : 524287);
+      if (isSetSuccess())
+        hashCode = hashCode * 8191 + success.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetTope()) ? 131071 : 524287);
+      if (isSetTope())
+        hashCode = hashCode * 8191 + tope.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(getTableProperties_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = java.lang.Boolean.compare(isSetSuccess(), 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.compare(isSetTope(), other.isSetTope());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetTope()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tope, other.tope);
+        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("getTableProperties_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("tope:");
+      if (this.tope == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.tope);
+      }
+      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 getTableProperties_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getTableProperties_resultStandardScheme getScheme() {
+        return new getTableProperties_resultStandardScheme();
+      }
+    }
+
+    private static class getTableProperties_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<getTableProperties_result> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, getTableProperties_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 0: // SUCCESS
+              if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+                {
+                  org.apache.thrift.protocol.TMap _map104 = iprot.readMapBegin();
+                  struct.success = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map104.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _key105;
+                  @org.apache.thrift.annotation.Nullable java.lang.String _val106;
+                  for (int _i107 = 0; _i107 < _map104.size; ++_i107)
+                  {
+                    _key105 = iprot.readString();
+                    _val106 = iprot.readString();
+                    struct.success.put(_key105, _val106);
+                  }
+                  iprot.readMapEnd();
+                }
+                struct.setSuccessIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 1: // TOPE
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.tope = new ThriftTableOperationException();
+                struct.tope.read(iprot);
+                struct.setTopeIsSet(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, getTableProperties_result struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.success != null) {
+          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+          {
+            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.success.size()));
+            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter108 : struct.success.entrySet())
+            {
+              oprot.writeString(_iter108.getKey());
+              oprot.writeString(_iter108.getValue());
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+        if (struct.tope != null) {
+          oprot.writeFieldBegin(TOPE_FIELD_DESC);
+          struct.tope.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class getTableProperties_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getTableProperties_resultTupleScheme getScheme() {
+        return new getTableProperties_resultTupleScheme();
+      }
+    }
+
+    private static class getTableProperties_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<getTableProperties_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, getTableProperties_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.isSetTope()) {
+          optionals.set(1);
+        }
+        oprot.writeBitSet(optionals, 2);
+        if (struct.isSetSuccess()) {
+          {
+            oprot.writeI32(struct.success.size());
+            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter109 : struct.success.entrySet())
+            {
+              oprot.writeString(_iter109.getKey());
+              oprot.writeString(_iter109.getValue());
+            }
+          }
+        }
+        if (struct.isSetTope()) {
+          struct.tope.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, getTableProperties_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)) {
+          {
+            org.apache.thrift.protocol.TMap _map110 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); 
+            struct.success = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map110.size);
+            @org.apache.thrift.annotation.Nullable java.lang.String _key111;
+            @org.apache.thrift.annotation.Nullable java.lang.String _val112;
+            for (int _i113 = 0; _i113 < _map110.size; ++_i113)
+            {
+              _key111 = iprot.readString();
+              _val112 = iprot.readString();
+              struct.success.put(_key111, _val112);
+            }
+          }
+          struct.setSuccessIsSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.tope = new ThriftTableOperationException();
+          struct.tope.read(iprot);
+          struct.setTopeIsSet(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 getVersionedTableProperties_args implements org.apache.thrift.TBase<getVersionedTableProperties_args, getVersionedTableProperties_args._Fields>, java.io.Serializable, Cloneable, Comparable<getVersionedTableProperties_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getVersionedTableProperties_args");
+
+    private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+    private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)3);
+    private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)2);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getVersionedTableProperties_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getVersionedTableProperties_argsTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
+    public @org.apache.thrift.annotation.Nullable java.lang.String 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 {
+      TINFO((short)1, "tinfo"),
+      CREDENTIALS((short)3, "credentials"),
+      TABLE_NAME((short)2, "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: // TINFO
+            return TINFO;
+          case 3: // CREDENTIALS
+            return CREDENTIALS;
+          case 2: // 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.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+      tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
+      tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getVersionedTableProperties_args.class, metaDataMap);
+    }
+
+    public getVersionedTableProperties_args() {
+    }
+
+    public getVersionedTableProperties_args(
+      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
+      java.lang.String tableName)
+    {
+      this();
+      this.tinfo = tinfo;
+      this.credentials = credentials;
+      this.tableName = tableName;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public getVersionedTableProperties_args(getVersionedTableProperties_args other) {
+      if (other.isSetTinfo()) {
+        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+      }
+      if (other.isSetCredentials()) {
+        this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
+      }
+      if (other.isSetTableName()) {
+        this.tableName = other.tableName;
+      }
+    }
+
+    public getVersionedTableProperties_args deepCopy() {
+      return new getVersionedTableProperties_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.tinfo = null;
+      this.credentials = null;
+      this.tableName = null;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+      return this.tinfo;
+    }
+
+    public getVersionedTableProperties_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+      this.tinfo = tinfo;
+      return this;
+    }
+
+    public void unsetTinfo() {
+      this.tinfo = null;
+    }
+
+    /** Returns true if field tinfo is set (has been assigned a value) and false otherwise */
+    public boolean isSetTinfo() {
+      return this.tinfo != null;
+    }
+
+    public void setTinfoIsSet(boolean value) {
+      if (!value) {
+        this.tinfo = null;
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.securityImpl.thrift.TCredentials getCredentials() {
+      return this.credentials;
+    }
+
+    public getVersionedTableProperties_args setCredentials(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) {
+      this.credentials = credentials;
+      return this;
+    }
+
+    public void unsetCredentials() {
+      this.credentials = null;
+    }
+
+    /** Returns true if field credentials is set (has been assigned a value) and false otherwise */
+    public boolean isSetCredentials() {
+      return this.credentials != null;
+    }
+
+    public void setCredentialsIsSet(boolean value) {
+      if (!value) {
+        this.credentials = null;
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.lang.String getTableName() {
+      return this.tableName;
+    }
+
+    public getVersionedTableProperties_args setTableName(@org.apache.thrift.annotation.Nullable java.lang.String 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 TINFO:
+        if (value == null) {
+          unsetTinfo();
+        } else {
+          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+        }
+        break;
+
+      case CREDENTIALS:
+        if (value == null) {
+          unsetCredentials();
+        } else {
+          setCredentials((org.apache.accumulo.core.securityImpl.thrift.TCredentials)value);
+        }
+        break;
+
+      case TABLE_NAME:
+        if (value == null) {
+          unsetTableName();
+        } else {
+          setTableName((java.lang.String)value);
+        }
+        break;
+
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case TINFO:
+        return getTinfo();
+
+      case CREDENTIALS:
+        return getCredentials();
+
+      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 TINFO:
+        return isSetTinfo();
+      case CREDENTIALS:
+        return isSetCredentials();
+      case TABLE_NAME:
+        return isSetTableName();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that instanceof getVersionedTableProperties_args)
+        return this.equals((getVersionedTableProperties_args)that);
+      return false;
+    }
+
+    public boolean equals(getVersionedTableProperties_args that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return true;
+
+      boolean this_present_tinfo = true && this.isSetTinfo();
+      boolean that_present_tinfo = true && that.isSetTinfo();
+      if (this_present_tinfo || that_present_tinfo) {
+        if (!(this_present_tinfo && that_present_tinfo))
+          return false;
+        if (!this.tinfo.equals(that.tinfo))
+          return false;
+      }
+
+      boolean this_present_credentials = true && this.isSetCredentials();
+      boolean that_present_credentials = true && that.isSetCredentials();
+      if (this_present_credentials || that_present_credentials) {
+        if (!(this_present_credentials && that_present_credentials))
+          return false;
+        if (!this.credentials.equals(that.credentials))
+          return false;
+      }
+
+      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 + ((isSetTinfo()) ? 131071 : 524287);
+      if (isSetTinfo())
+        hashCode = hashCode * 8191 + tinfo.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetCredentials()) ? 131071 : 524287);
+      if (isSetCredentials())
+        hashCode = hashCode * 8191 + credentials.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetTableName()) ? 131071 : 524287);
+      if (isSetTableName())
+        hashCode = hashCode * 8191 + tableName.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(getVersionedTableProperties_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = java.lang.Boolean.compare(isSetTinfo(), other.isSetTinfo());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetTinfo()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tinfo, other.tinfo);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.compare(isSetCredentials(), other.isSetCredentials());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetCredentials()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.credentials, other.credentials);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.compare(isSetTableName(), 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("getVersionedTableProperties_args(");
+      boolean first = true;
+
+      sb.append("tinfo:");
+      if (this.tinfo == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.tinfo);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("credentials:");
+      if (this.credentials == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.credentials);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      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
+      // check for sub-struct validity
+      if (tinfo != null) {
+        tinfo.validate();
+      }
+      if (credentials != null) {
+        credentials.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 getVersionedTableProperties_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getVersionedTableProperties_argsStandardScheme getScheme() {
+        return new getVersionedTableProperties_argsStandardScheme();
+      }
+    }
+
+    private static class getVersionedTableProperties_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<getVersionedTableProperties_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, getVersionedTableProperties_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: // TINFO
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+                struct.tinfo.read(iprot);
+                struct.setTinfoIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 3: // CREDENTIALS
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
+                struct.credentials.read(iprot);
+                struct.setCredentialsIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 2: // TABLE_NAME
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.tableName = iprot.readString();
+                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, getVersionedTableProperties_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.tinfo != null) {
+          oprot.writeFieldBegin(TINFO_FIELD_DESC);
+          struct.tinfo.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.tableName != null) {
+          oprot.writeFieldBegin(TABLE_NAME_FIELD_DESC);
+          oprot.writeString(struct.tableName);
+          oprot.writeFieldEnd();
+        }
+        if (struct.credentials != null) {
+          oprot.writeFieldBegin(CREDENTIALS_FIELD_DESC);
+          struct.credentials.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class getVersionedTableProperties_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getVersionedTableProperties_argsTupleScheme getScheme() {
+        return new getVersionedTableProperties_argsTupleScheme();
+      }
+    }
+
+    private static class getVersionedTableProperties_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<getVersionedTableProperties_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, getVersionedTableProperties_args 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.isSetTinfo()) {
+          optionals.set(0);
+        }
+        if (struct.isSetCredentials()) {
+          optionals.set(1);
+        }
+        if (struct.isSetTableName()) {
+          optionals.set(2);
+        }
+        oprot.writeBitSet(optionals, 3);
+        if (struct.isSetTinfo()) {
+          struct.tinfo.write(oprot);
+        }
+        if (struct.isSetCredentials()) {
+          struct.credentials.write(oprot);
+        }
+        if (struct.isSetTableName()) {
+          oprot.writeString(struct.tableName);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, getVersionedTableProperties_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet incoming = iprot.readBitSet(3);
+        if (incoming.get(0)) {
+          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+          struct.tinfo.read(iprot);
+          struct.setTinfoIsSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
+          struct.credentials.read(iprot);
+          struct.setCredentialsIsSet(true);
+        }
+        if (incoming.get(2)) {
+          struct.tableName = iprot.readString();
+          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 getVersionedTableProperties_result implements org.apache.thrift.TBase<getVersionedTableProperties_result, getVersionedTableProperties_result._Fields>, java.io.Serializable, Cloneable, Comparable<getVersionedTableProperties_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getVersionedTableProperties_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 TOPE_FIELD_DESC = new org.apache.thrift.protocol.TField("tope", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getVersionedTableProperties_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getVersionedTableProperties_resultTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable TVersionedProperties success; // required
+    public @org.apache.thrift.annotation.Nullable ThriftTableOperationException tope; // 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"),
+      TOPE((short)1, "tope");
+
+      private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
+
+      static {
+        for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      @org.apache.thrift.annotation.Nullable
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 0: // SUCCESS
+            return SUCCESS;
+          case 1: // TOPE
+            return TOPE;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      @org.apache.thrift.annotation.Nullable
+      public static _Fields findByName(java.lang.String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final java.lang.String _fieldName;
+
+      _Fields(short thriftId, java.lang.String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public java.lang.String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TVersionedProperties.class)));
       tmpMap.put(_Fields.TOPE, new org.apache.thrift.meta_data.FieldMetaData("tope", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ThriftTableOperationException.class)));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(revokeNamespacePermission_result.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getVersionedTableProperties_result.class, metaDataMap);
     }
 
-    public revokeNamespacePermission_result() {
+    public getVersionedTableProperties_result() {
     }
 
-    public revokeNamespacePermission_result(
-      ThriftSecurityException sec,
+    public getVersionedTableProperties_result(
+      TVersionedProperties success,
       ThriftTableOperationException tope)
     {
       this();
-      this.sec = sec;
+      this.success = success;
       this.tope = tope;
     }
 
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public revokeNamespacePermission_result(revokeNamespacePermission_result other) {
-      if (other.isSetSec()) {
-        this.sec = new ThriftSecurityException(other.sec);
+    public getVersionedTableProperties_result(getVersionedTableProperties_result other) {
+      if (other.isSetSuccess()) {
+        this.success = new TVersionedProperties(other.success);
       }
       if (other.isSetTope()) {
         this.tope = new ThriftTableOperationException(other.tope);
       }
     }
 
-    public revokeNamespacePermission_result deepCopy() {
-      return new revokeNamespacePermission_result(this);
+    public getVersionedTableProperties_result deepCopy() {
+      return new getVersionedTableProperties_result(this);
     }
 
     @Override
     public void clear() {
-      this.sec = null;
+      this.success = null;
       this.tope = null;
     }
 
     @org.apache.thrift.annotation.Nullable
-    public ThriftSecurityException getSec() {
-      return this.sec;
+    public TVersionedProperties getSuccess() {
+      return this.success;
     }
 
-    public revokeNamespacePermission_result setSec(@org.apache.thrift.annotation.Nullable ThriftSecurityException sec) {
-      this.sec = sec;
+    public getVersionedTableProperties_result setSuccess(@org.apache.thrift.annotation.Nullable TVersionedProperties success) {
+      this.success = success;
       return this;
     }
 
-    public void unsetSec() {
-      this.sec = null;
+    public void unsetSuccess() {
+      this.success = null;
     }
 
-    /** Returns true if field sec is set (has been assigned a value) and false otherwise */
-    public boolean isSetSec() {
-      return this.sec != 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 setSecIsSet(boolean value) {
+    public void setSuccessIsSet(boolean value) {
       if (!value) {
-        this.sec = null;
+        this.success = null;
       }
     }
 
@@ -30421,7 +37429,7 @@ public class ClientService {
       return this.tope;
     }
 
-    public revokeNamespacePermission_result setTope(@org.apache.thrift.annotation.Nullable ThriftTableOperationException tope) {
+    public getVersionedTableProperties_result setTope(@org.apache.thrift.annotation.Nullable ThriftTableOperationException tope) {
       this.tope = tope;
       return this;
     }
@@ -30443,11 +37451,11 @@ public class ClientService {
 
     public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
       switch (field) {
-      case SEC:
+      case SUCCESS:
         if (value == null) {
-          unsetSec();
+          unsetSuccess();
         } else {
-          setSec((ThriftSecurityException)value);
+          setSuccess((TVersionedProperties)value);
         }
         break;
 
@@ -30465,8 +37473,8 @@ public class ClientService {
     @org.apache.thrift.annotation.Nullable
     public java.lang.Object getFieldValue(_Fields field) {
       switch (field) {
-      case SEC:
-        return getSec();
+      case SUCCESS:
+        return getSuccess();
 
       case TOPE:
         return getTope();
@@ -30482,8 +37490,8 @@ public class ClientService {
       }
 
       switch (field) {
-      case SEC:
-        return isSetSec();
+      case SUCCESS:
+        return isSetSuccess();
       case TOPE:
         return isSetTope();
       }
@@ -30492,23 +37500,23 @@ public class ClientService {
 
     @Override
     public boolean equals(java.lang.Object that) {
-      if (that instanceof revokeNamespacePermission_result)
-        return this.equals((revokeNamespacePermission_result)that);
+      if (that instanceof getVersionedTableProperties_result)
+        return this.equals((getVersionedTableProperties_result)that);
       return false;
     }
 
-    public boolean equals(revokeNamespacePermission_result that) {
+    public boolean equals(getVersionedTableProperties_result that) {
       if (that == null)
         return false;
       if (this == that)
         return true;
 
-      boolean this_present_sec = true && this.isSetSec();
-      boolean that_present_sec = true && that.isSetSec();
-      if (this_present_sec || that_present_sec) {
-        if (!(this_present_sec && that_present_sec))
+      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.sec.equals(that.sec))
+        if (!this.success.equals(that.success))
           return false;
       }
 
@@ -30528,9 +37536,9 @@ public class ClientService {
     public int hashCode() {
       int hashCode = 1;
 
-      hashCode = hashCode * 8191 + ((isSetSec()) ? 131071 : 524287);
-      if (isSetSec())
-        hashCode = hashCode * 8191 + sec.hashCode();
+      hashCode = hashCode * 8191 + ((isSetSuccess()) ? 131071 : 524287);
+      if (isSetSuccess())
+        hashCode = hashCode * 8191 + success.hashCode();
 
       hashCode = hashCode * 8191 + ((isSetTope()) ? 131071 : 524287);
       if (isSetTope())
@@ -30540,19 +37548,19 @@ public class ClientService {
     }
 
     @Override
-    public int compareTo(revokeNamespacePermission_result other) {
+    public int compareTo(getVersionedTableProperties_result other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
 
       int lastComparison = 0;
 
-      lastComparison = java.lang.Boolean.compare(isSetSec(), other.isSetSec());
+      lastComparison = java.lang.Boolean.compare(isSetSuccess(), other.isSetSuccess());
       if (lastComparison != 0) {
         return lastComparison;
       }
-      if (isSetSec()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sec, other.sec);
+      if (isSetSuccess()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success);
         if (lastComparison != 0) {
           return lastComparison;
         }
@@ -30585,14 +37593,14 @@ public class ClientService {
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("revokeNamespacePermission_result(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("getVersionedTableProperties_result(");
       boolean first = true;
 
-      sb.append("sec:");
-      if (this.sec == null) {
+      sb.append("success:");
+      if (this.success == null) {
         sb.append("null");
       } else {
-        sb.append(this.sec);
+        sb.append(this.success);
       }
       first = false;
       if (!first) sb.append(", ");
@@ -30610,6 +37618,9 @@ public class ClientService {
     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 {
@@ -30628,15 +37639,15 @@ public class ClientService {
       }
     }
 
-    private static class revokeNamespacePermission_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public revokeNamespacePermission_resultStandardScheme getScheme() {
-        return new revokeNamespacePermission_resultStandardScheme();
+    private static class getVersionedTableProperties_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getVersionedTableProperties_resultStandardScheme getScheme() {
+        return new getVersionedTableProperties_resultStandardScheme();
       }
     }
 
-    private static class revokeNamespacePermission_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<revokeNamespacePermission_result> {
+    private static class getVersionedTableProperties_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<getVersionedTableProperties_result> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, revokeNamespacePermission_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, getVersionedTableProperties_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -30646,16 +37657,16 @@ public class ClientService {
             break;
           }
           switch (schemeField.id) {
-            case 1: // SEC
+            case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.sec = new ThriftSecurityException();
-                struct.sec.read(iprot);
-                struct.setSecIsSet(true);
+                struct.success = new TVersionedProperties();
+                struct.success.read(iprot);
+                struct.setSuccessIsSet(true);
               } else { 
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
               break;
-            case 2: // TOPE
+            case 1: // TOPE
               if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
                 struct.tope = new ThriftTableOperationException();
                 struct.tope.read(iprot);
@@ -30675,13 +37686,13 @@ public class ClientService {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, revokeNamespacePermission_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, getVersionedTableProperties_result struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
-        if (struct.sec != null) {
-          oprot.writeFieldBegin(SEC_FIELD_DESC);
-          struct.sec.write(oprot);
+        if (struct.success != null) {
+          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+          struct.success.write(oprot);
           oprot.writeFieldEnd();
         }
         if (struct.tope != null) {
@@ -30695,27 +37706,27 @@ public class ClientService {
 
     }
 
-    private static class revokeNamespacePermission_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public revokeNamespacePermission_resultTupleScheme getScheme() {
-        return new revokeNamespacePermission_resultTupleScheme();
+    private static class getVersionedTableProperties_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getVersionedTableProperties_resultTupleScheme getScheme() {
+        return new getVersionedTableProperties_resultTupleScheme();
       }
     }
 
-    private static class revokeNamespacePermission_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<revokeNamespacePermission_result> {
+    private static class getVersionedTableProperties_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<getVersionedTableProperties_result> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, revokeNamespacePermission_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, getVersionedTableProperties_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.isSetSec()) {
+        if (struct.isSetSuccess()) {
           optionals.set(0);
         }
         if (struct.isSetTope()) {
           optionals.set(1);
         }
         oprot.writeBitSet(optionals, 2);
-        if (struct.isSetSec()) {
-          struct.sec.write(oprot);
+        if (struct.isSetSuccess()) {
+          struct.success.write(oprot);
         }
         if (struct.isSetTope()) {
           struct.tope.write(oprot);
@@ -30723,13 +37734,13 @@ public class ClientService {
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, revokeNamespacePermission_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, getVersionedTableProperties_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.sec = new ThriftSecurityException();
-          struct.sec.read(iprot);
-          struct.setSecIsSet(true);
+          struct.success = new TVersionedProperties();
+          struct.success.read(iprot);
+          struct.setSuccessIsSet(true);
         }
         if (incoming.get(1)) {
           struct.tope = new ThriftTableOperationException();
@@ -30744,33 +37755,25 @@ public class ClientService {
     }
   }
 
-  public static class getConfiguration_args implements org.apache.thrift.TBase<getConfiguration_args, getConfiguration_args._Fields>, java.io.Serializable, Cloneable, Comparable<getConfiguration_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getConfiguration_args");
+  public static class getNamespaceConfiguration_args implements org.apache.thrift.TBase<getNamespaceConfiguration_args, getNamespaceConfiguration_args._Fields>, java.io.Serializable, Cloneable, Comparable<getNamespaceConfiguration_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getNamespaceConfiguration_args");
 
-    private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)2);
-    private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)3);
-    private static final org.apache.thrift.protocol.TField TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("type", org.apache.thrift.protocol.TType.I32, (short)1);
+    private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+    private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+    private static final org.apache.thrift.protocol.TField NS_FIELD_DESC = new org.apache.thrift.protocol.TField("ns", org.apache.thrift.protocol.TType.STRING, (short)3);
 
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getConfiguration_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getConfiguration_argsTupleSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getNamespaceConfiguration_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getNamespaceConfiguration_argsTupleSchemeFactory();
 
     public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
     public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
-    /**
-     * 
-     * @see ConfigurationType
-     */
-    public @org.apache.thrift.annotation.Nullable ConfigurationType type; // required
+    public @org.apache.thrift.annotation.Nullable java.lang.String ns; // 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 {
-      TINFO((short)2, "tinfo"),
-      CREDENTIALS((short)3, "credentials"),
-      /**
-       * 
-       * @see ConfigurationType
-       */
-      TYPE((short)1, "type");
+      TINFO((short)1, "tinfo"),
+      CREDENTIALS((short)2, "credentials"),
+      NS((short)3, "ns");
 
       private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
 
@@ -30786,12 +37789,12 @@ public class ClientService {
       @org.apache.thrift.annotation.Nullable
       public static _Fields findByThriftId(int fieldId) {
         switch(fieldId) {
-          case 2: // TINFO
+          case 1: // TINFO
             return TINFO;
-          case 3: // CREDENTIALS
+          case 2: // CREDENTIALS
             return CREDENTIALS;
-          case 1: // TYPE
-            return TYPE;
+          case 3: // NS
+            return NS;
           default:
             return null;
         }
@@ -30840,50 +37843,50 @@ public class ClientService {
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
       tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
-      tmpMap.put(_Fields.TYPE, new org.apache.thrift.meta_data.FieldMetaData("type", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, ConfigurationType.class)));
+      tmpMap.put(_Fields.NS, new org.apache.thrift.meta_data.FieldMetaData("ns", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getConfiguration_args.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getNamespaceConfiguration_args.class, metaDataMap);
     }
 
-    public getConfiguration_args() {
+    public getNamespaceConfiguration_args() {
     }
 
-    public getConfiguration_args(
+    public getNamespaceConfiguration_args(
       org.apache.accumulo.core.trace.thrift.TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
-      ConfigurationType type)
+      java.lang.String ns)
     {
       this();
       this.tinfo = tinfo;
       this.credentials = credentials;
-      this.type = type;
+      this.ns = ns;
     }
 
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public getConfiguration_args(getConfiguration_args other) {
+    public getNamespaceConfiguration_args(getNamespaceConfiguration_args other) {
       if (other.isSetTinfo()) {
         this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
       }
-      if (other.isSetType()) {
-        this.type = other.type;
+      if (other.isSetNs()) {
+        this.ns = other.ns;
       }
     }
 
-    public getConfiguration_args deepCopy() {
-      return new getConfiguration_args(this);
+    public getNamespaceConfiguration_args deepCopy() {
+      return new getNamespaceConfiguration_args(this);
     }
 
     @Override
     public void clear() {
       this.tinfo = null;
       this.credentials = null;
-      this.type = null;
+      this.ns = null;
     }
 
     @org.apache.thrift.annotation.Nullable
@@ -30891,7 +37894,7 @@ public class ClientService {
       return this.tinfo;
     }
 
-    public getConfiguration_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public getNamespaceConfiguration_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -30916,7 +37919,7 @@ public class ClientService {
       return this.credentials;
     }
 
-    public getConfiguration_args setCredentials(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) {
+    public getNamespaceConfiguration_args setCredentials(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) {
       this.credentials = credentials;
       return this;
     }
@@ -30936,36 +37939,28 @@ public class ClientService {
       }
     }
 
-    /**
-     * 
-     * @see ConfigurationType
-     */
     @org.apache.thrift.annotation.Nullable
-    public ConfigurationType getType() {
-      return this.type;
+    public java.lang.String getNs() {
+      return this.ns;
     }
 
-    /**
-     * 
-     * @see ConfigurationType
-     */
-    public getConfiguration_args setType(@org.apache.thrift.annotation.Nullable ConfigurationType type) {
-      this.type = type;
+    public getNamespaceConfiguration_args setNs(@org.apache.thrift.annotation.Nullable java.lang.String ns) {
+      this.ns = ns;
       return this;
     }
 
-    public void unsetType() {
-      this.type = null;
+    public void unsetNs() {
+      this.ns = null;
     }
 
-    /** Returns true if field type is set (has been assigned a value) and false otherwise */
-    public boolean isSetType() {
-      return this.type != null;
+    /** Returns true if field ns is set (has been assigned a value) and false otherwise */
+    public boolean isSetNs() {
+      return this.ns != null;
     }
 
-    public void setTypeIsSet(boolean value) {
+    public void setNsIsSet(boolean value) {
       if (!value) {
-        this.type = null;
+        this.ns = null;
       }
     }
 
@@ -30987,11 +37982,11 @@ public class ClientService {
         }
         break;
 
-      case TYPE:
+      case NS:
         if (value == null) {
-          unsetType();
+          unsetNs();
         } else {
-          setType((ConfigurationType)value);
+          setNs((java.lang.String)value);
         }
         break;
 
@@ -31007,8 +38002,8 @@ public class ClientService {
       case CREDENTIALS:
         return getCredentials();
 
-      case TYPE:
-        return getType();
+      case NS:
+        return getNs();
 
       }
       throw new java.lang.IllegalStateException();
@@ -31025,20 +38020,20 @@ public class ClientService {
         return isSetTinfo();
       case CREDENTIALS:
         return isSetCredentials();
-      case TYPE:
-        return isSetType();
+      case NS:
+        return isSetNs();
       }
       throw new java.lang.IllegalStateException();
     }
 
     @Override
     public boolean equals(java.lang.Object that) {
-      if (that instanceof getConfiguration_args)
-        return this.equals((getConfiguration_args)that);
+      if (that instanceof getNamespaceConfiguration_args)
+        return this.equals((getNamespaceConfiguration_args)that);
       return false;
     }
 
-    public boolean equals(getConfiguration_args that) {
+    public boolean equals(getNamespaceConfiguration_args that) {
       if (that == null)
         return false;
       if (this == that)
@@ -31062,12 +38057,12 @@ public class ClientService {
           return false;
       }
 
-      boolean this_present_type = true && this.isSetType();
-      boolean that_present_type = true && that.isSetType();
-      if (this_present_type || that_present_type) {
-        if (!(this_present_type && that_present_type))
+      boolean this_present_ns = true && this.isSetNs();
+      boolean that_present_ns = true && that.isSetNs();
+      if (this_present_ns || that_present_ns) {
+        if (!(this_present_ns && that_present_ns))
           return false;
-        if (!this.type.equals(that.type))
+        if (!this.ns.equals(that.ns))
           return false;
       }
 
@@ -31086,15 +38081,15 @@ public class ClientService {
       if (isSetCredentials())
         hashCode = hashCode * 8191 + credentials.hashCode();
 
-      hashCode = hashCode * 8191 + ((isSetType()) ? 131071 : 524287);
-      if (isSetType())
-        hashCode = hashCode * 8191 + type.getValue();
+      hashCode = hashCode * 8191 + ((isSetNs()) ? 131071 : 524287);
+      if (isSetNs())
+        hashCode = hashCode * 8191 + ns.hashCode();
 
       return hashCode;
     }
 
     @Override
-    public int compareTo(getConfiguration_args other) {
+    public int compareTo(getNamespaceConfiguration_args other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -31121,12 +38116,12 @@ public class ClientService {
           return lastComparison;
         }
       }
-      lastComparison = java.lang.Boolean.compare(isSetType(), other.isSetType());
+      lastComparison = java.lang.Boolean.compare(isSetNs(), other.isSetNs());
       if (lastComparison != 0) {
         return lastComparison;
       }
-      if (isSetType()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.type, other.type);
+      if (isSetNs()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.ns, other.ns);
         if (lastComparison != 0) {
           return lastComparison;
         }
@@ -31149,7 +38144,7 @@ public class ClientService {
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("getConfiguration_args(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("getNamespaceConfiguration_args(");
       boolean first = true;
 
       sb.append("tinfo:");
@@ -31168,11 +38163,11 @@ public class ClientService {
       }
       first = false;
       if (!first) sb.append(", ");
-      sb.append("type:");
-      if (this.type == null) {
+      sb.append("ns:");
+      if (this.ns == null) {
         sb.append("null");
       } else {
-        sb.append(this.type);
+        sb.append(this.ns);
       }
       first = false;
       sb.append(")");
@@ -31206,15 +38201,15 @@ public class ClientService {
       }
     }
 
-    private static class getConfiguration_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public getConfiguration_argsStandardScheme getScheme() {
-        return new getConfiguration_argsStandardScheme();
+    private static class getNamespaceConfiguration_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getNamespaceConfiguration_argsStandardScheme getScheme() {
+        return new getNamespaceConfiguration_argsStandardScheme();
       }
     }
 
-    private static class getConfiguration_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<getConfiguration_args> {
+    private static class getNamespaceConfiguration_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<getNamespaceConfiguration_args> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, getConfiguration_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, getNamespaceConfiguration_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -31224,7 +38219,7 @@ public class ClientService {
             break;
           }
           switch (schemeField.id) {
-            case 2: // TINFO
+            case 1: // TINFO
               if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
                 struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
                 struct.tinfo.read(iprot);
@@ -31233,7 +38228,7 @@ public class ClientService {
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
               break;
-            case 3: // CREDENTIALS
+            case 2: // CREDENTIALS
               if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
                 struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
                 struct.credentials.read(iprot);
@@ -31242,10 +38237,10 @@ public class ClientService {
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
               break;
-            case 1: // TYPE
-              if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
-                struct.type = org.apache.accumulo.core.clientImpl.thrift.ConfigurationType.findByValue(iprot.readI32());
-                struct.setTypeIsSet(true);
+            case 3: // NS
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.ns = iprot.readString();
+                struct.setNsIsSet(true);
               } else { 
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
@@ -31261,15 +38256,10 @@ public class ClientService {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, getConfiguration_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, getNamespaceConfiguration_args struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
-        if (struct.type != null) {
-          oprot.writeFieldBegin(TYPE_FIELD_DESC);
-          oprot.writeI32(struct.type.getValue());
-          oprot.writeFieldEnd();
-        }
         if (struct.tinfo != null) {
           oprot.writeFieldBegin(TINFO_FIELD_DESC);
           struct.tinfo.write(oprot);
@@ -31280,22 +38270,27 @@ public class ClientService {
           struct.credentials.write(oprot);
           oprot.writeFieldEnd();
         }
+        if (struct.ns != null) {
+          oprot.writeFieldBegin(NS_FIELD_DESC);
+          oprot.writeString(struct.ns);
+          oprot.writeFieldEnd();
+        }
         oprot.writeFieldStop();
         oprot.writeStructEnd();
       }
 
     }
 
-    private static class getConfiguration_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public getConfiguration_argsTupleScheme getScheme() {
-        return new getConfiguration_argsTupleScheme();
+    private static class getNamespaceConfiguration_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getNamespaceConfiguration_argsTupleScheme getScheme() {
+        return new getNamespaceConfiguration_argsTupleScheme();
       }
     }
 
-    private static class getConfiguration_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<getConfiguration_args> {
+    private static class getNamespaceConfiguration_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<getNamespaceConfiguration_args> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, getConfiguration_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, getNamespaceConfiguration_args 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.isSetTinfo()) {
@@ -31304,7 +38299,7 @@ public class ClientService {
         if (struct.isSetCredentials()) {
           optionals.set(1);
         }
-        if (struct.isSetType()) {
+        if (struct.isSetNs()) {
           optionals.set(2);
         }
         oprot.writeBitSet(optionals, 3);
@@ -31314,13 +38309,13 @@ public class ClientService {
         if (struct.isSetCredentials()) {
           struct.credentials.write(oprot);
         }
-        if (struct.isSetType()) {
-          oprot.writeI32(struct.type.getValue());
+        if (struct.isSetNs()) {
+          oprot.writeString(struct.ns);
         }
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, getConfiguration_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, getNamespaceConfiguration_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet incoming = iprot.readBitSet(3);
         if (incoming.get(0)) {
@@ -31334,8 +38329,8 @@ public class ClientService {
           struct.setCredentialsIsSet(true);
         }
         if (incoming.get(2)) {
-          struct.type = org.apache.accumulo.core.clientImpl.thrift.ConfigurationType.findByValue(iprot.readI32());
-          struct.setTypeIsSet(true);
+          struct.ns = iprot.readString();
+          struct.setNsIsSet(true);
         }
       }
     }
@@ -31345,19 +38340,22 @@ public class ClientService {
     }
   }
 
-  public static class getConfiguration_result implements org.apache.thrift.TBase<getConfiguration_result, getConfiguration_result._Fields>, java.io.Serializable, Cloneable, Comparable<getConfiguration_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getConfiguration_result");
+  public static class getNamespaceConfiguration_result implements org.apache.thrift.TBase<getNamespaceConfiguration_result, getNamespaceConfiguration_result._Fields>, java.io.Serializable, Cloneable, Comparable<getNamespaceConfiguration_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getNamespaceConfiguration_result");
 
     private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.MAP, (short)0);
+    private static final org.apache.thrift.protocol.TField TOPE_FIELD_DESC = new org.apache.thrift.protocol.TField("tope", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getConfiguration_resultStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getConfiguration_resultTupleSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getNamespaceConfiguration_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getNamespaceConfiguration_resultTupleSchemeFactory();
 
     public @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,java.lang.String> success; // required
+    public @org.apache.thrift.annotation.Nullable ThriftTableOperationException tope; // 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");
+      SUCCESS((short)0, "success"),
+      TOPE((short)1, "tope");
 
       private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
 
@@ -31375,6 +38373,8 @@ public class ClientService {
         switch(fieldId) {
           case 0: // SUCCESS
             return SUCCESS;
+          case 1: // TOPE
+            return TOPE;
           default:
             return null;
         }
@@ -31423,37 +38423,45 @@ public class ClientService {
           new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
               new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
               new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+      tmpMap.put(_Fields.TOPE, new org.apache.thrift.meta_data.FieldMetaData("tope", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ThriftTableOperationException.class)));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getConfiguration_result.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getNamespaceConfiguration_result.class, metaDataMap);
     }
 
-    public getConfiguration_result() {
+    public getNamespaceConfiguration_result() {
     }
 
-    public getConfiguration_result(
-      java.util.Map<java.lang.String,java.lang.String> success)
+    public getNamespaceConfiguration_result(
+      java.util.Map<java.lang.String,java.lang.String> success,
+      ThriftTableOperationException tope)
     {
       this();
       this.success = success;
+      this.tope = tope;
     }
 
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public getConfiguration_result(getConfiguration_result other) {
+    public getNamespaceConfiguration_result(getNamespaceConfiguration_result other) {
       if (other.isSetSuccess()) {
         java.util.Map<java.lang.String,java.lang.String> __this__success = new java.util.HashMap<java.lang.String,java.lang.String>(other.success);
         this.success = __this__success;
       }
+      if (other.isSetTope()) {
+        this.tope = new ThriftTableOperationException(other.tope);
+      }
     }
 
-    public getConfiguration_result deepCopy() {
-      return new getConfiguration_result(this);
+    public getNamespaceConfiguration_result deepCopy() {
+      return new getNamespaceConfiguration_result(this);
     }
 
     @Override
     public void clear() {
       this.success = null;
+      this.tope = null;
     }
 
     public int getSuccessSize() {
@@ -31472,7 +38480,7 @@ public class ClientService {
       return this.success;
     }
 
-    public getConfiguration_result setSuccess(@org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,java.lang.String> success) {
+    public getNamespaceConfiguration_result setSuccess(@org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,java.lang.String> success) {
       this.success = success;
       return this;
     }
@@ -31492,6 +38500,31 @@ public class ClientService {
       }
     }
 
+    @org.apache.thrift.annotation.Nullable
+    public ThriftTableOperationException getTope() {
+      return this.tope;
+    }
+
+    public getNamespaceConfiguration_result setTope(@org.apache.thrift.annotation.Nullable ThriftTableOperationException tope) {
+      this.tope = tope;
+      return this;
+    }
+
+    public void unsetTope() {
+      this.tope = null;
+    }
+
+    /** Returns true if field tope is set (has been assigned a value) and false otherwise */
+    public boolean isSetTope() {
+      return this.tope != null;
+    }
+
+    public void setTopeIsSet(boolean value) {
+      if (!value) {
+        this.tope = null;
+      }
+    }
+
     public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
       switch (field) {
       case SUCCESS:
@@ -31502,6 +38535,14 @@ public class ClientService {
         }
         break;
 
+      case TOPE:
+        if (value == null) {
+          unsetTope();
+        } else {
+          setTope((ThriftTableOperationException)value);
+        }
+        break;
+
       }
     }
 
@@ -31511,6 +38552,9 @@ public class ClientService {
       case SUCCESS:
         return getSuccess();
 
+      case TOPE:
+        return getTope();
+
       }
       throw new java.lang.IllegalStateException();
     }
@@ -31524,18 +38568,20 @@ public class ClientService {
       switch (field) {
       case SUCCESS:
         return isSetSuccess();
+      case TOPE:
+        return isSetTope();
       }
       throw new java.lang.IllegalStateException();
     }
 
     @Override
     public boolean equals(java.lang.Object that) {
-      if (that instanceof getConfiguration_result)
-        return this.equals((getConfiguration_result)that);
+      if (that instanceof getNamespaceConfiguration_result)
+        return this.equals((getNamespaceConfiguration_result)that);
       return false;
     }
 
-    public boolean equals(getConfiguration_result that) {
+    public boolean equals(getNamespaceConfiguration_result that) {
       if (that == null)
         return false;
       if (this == that)
@@ -31550,6 +38596,15 @@ public class ClientService {
           return false;
       }
 
+      boolean this_present_tope = true && this.isSetTope();
+      boolean that_present_tope = true && that.isSetTope();
+      if (this_present_tope || that_present_tope) {
+        if (!(this_present_tope && that_present_tope))
+          return false;
+        if (!this.tope.equals(that.tope))
+          return false;
+      }
+
       return true;
     }
 
@@ -31561,11 +38616,15 @@ public class ClientService {
       if (isSetSuccess())
         hashCode = hashCode * 8191 + success.hashCode();
 
+      hashCode = hashCode * 8191 + ((isSetTope()) ? 131071 : 524287);
+      if (isSetTope())
+        hashCode = hashCode * 8191 + tope.hashCode();
+
       return hashCode;
     }
 
     @Override
-    public int compareTo(getConfiguration_result other) {
+    public int compareTo(getNamespaceConfiguration_result other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -31582,6 +38641,16 @@ public class ClientService {
           return lastComparison;
         }
       }
+      lastComparison = java.lang.Boolean.compare(isSetTope(), other.isSetTope());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetTope()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tope, other.tope);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
       return 0;
     }
 
@@ -31600,7 +38669,7 @@ public class ClientService {
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("getConfiguration_result(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("getNamespaceConfiguration_result(");
       boolean first = true;
 
       sb.append("success:");
@@ -31610,6 +38679,14 @@ public class ClientService {
         sb.append(this.success);
       }
       first = false;
+      if (!first) sb.append(", ");
+      sb.append("tope:");
+      if (this.tope == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.tope);
+      }
+      first = false;
       sb.append(")");
       return sb.toString();
     }
@@ -31635,15 +38712,15 @@ public class ClientService {
       }
     }
 
-    private static class getConfiguration_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public getConfiguration_resultStandardScheme getScheme() {
-        return new getConfiguration_resultStandardScheme();
+    private static class getNamespaceConfiguration_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getNamespaceConfiguration_resultStandardScheme getScheme() {
+        return new getNamespaceConfiguration_resultStandardScheme();
       }
     }
 
-    private static class getConfiguration_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<getConfiguration_result> {
+    private static class getNamespaceConfiguration_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<getNamespaceConfiguration_result> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, getConfiguration_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, getNamespaceConfiguration_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -31656,15 +38733,15 @@ public class ClientService {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map64 = iprot.readMapBegin();
-                  struct.success = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map64.size);
-                  @org.apache.thrift.annotation.Nullable java.lang.String _key65;
-                  @org.apache.thrift.annotation.Nullable java.lang.String _val66;
-                  for (int _i67 = 0; _i67 < _map64.size; ++_i67)
+                  org.apache.thrift.protocol.TMap _map114 = iprot.readMapBegin();
+                  struct.success = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map114.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _key115;
+                  @org.apache.thrift.annotation.Nullable java.lang.String _val116;
+                  for (int _i117 = 0; _i117 < _map114.size; ++_i117)
                   {
-                    _key65 = iprot.readString();
-                    _val66 = iprot.readString();
-                    struct.success.put(_key65, _val66);
+                    _key115 = iprot.readString();
+                    _val116 = iprot.readString();
+                    struct.success.put(_key115, _val116);
                   }
                   iprot.readMapEnd();
                 }
@@ -31673,6 +38750,15 @@ public class ClientService {
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
               break;
+            case 1: // TOPE
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.tope = new ThriftTableOperationException();
+                struct.tope.read(iprot);
+                struct.setTopeIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
             default:
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
           }
@@ -31684,7 +38770,7 @@ public class ClientService {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, getConfiguration_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, getNamespaceConfiguration_result struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -31692,68 +38778,84 @@ public class ClientService {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter68 : struct.success.entrySet())
+            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter118 : struct.success.entrySet())
             {
-              oprot.writeString(_iter68.getKey());
-              oprot.writeString(_iter68.getValue());
+              oprot.writeString(_iter118.getKey());
+              oprot.writeString(_iter118.getValue());
             }
             oprot.writeMapEnd();
           }
           oprot.writeFieldEnd();
         }
+        if (struct.tope != null) {
+          oprot.writeFieldBegin(TOPE_FIELD_DESC);
+          struct.tope.write(oprot);
+          oprot.writeFieldEnd();
+        }
         oprot.writeFieldStop();
         oprot.writeStructEnd();
       }
 
     }
 
-    private static class getConfiguration_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public getConfiguration_resultTupleScheme getScheme() {
-        return new getConfiguration_resultTupleScheme();
+    private static class getNamespaceConfiguration_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getNamespaceConfiguration_resultTupleScheme getScheme() {
+        return new getNamespaceConfiguration_resultTupleScheme();
       }
     }
 
-    private static class getConfiguration_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<getConfiguration_result> {
+    private static class getNamespaceConfiguration_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<getNamespaceConfiguration_result> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, getConfiguration_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, getNamespaceConfiguration_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet optionals = new java.util.BitSet();
         if (struct.isSetSuccess()) {
           optionals.set(0);
         }
-        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetTope()) {
+          optionals.set(1);
+        }
+        oprot.writeBitSet(optionals, 2);
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter69 : struct.success.entrySet())
+            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter119 : struct.success.entrySet())
             {
-              oprot.writeString(_iter69.getKey());
-              oprot.writeString(_iter69.getValue());
+              oprot.writeString(_iter119.getKey());
+              oprot.writeString(_iter119.getValue());
             }
           }
         }
+        if (struct.isSetTope()) {
+          struct.tope.write(oprot);
+        }
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, getConfiguration_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, getNamespaceConfiguration_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)) {
           {
-            org.apache.thrift.protocol.TMap _map70 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); 
-            struct.success = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map70.size);
-            @org.apache.thrift.annotation.Nullable java.lang.String _key71;
-            @org.apache.thrift.annotation.Nullable java.lang.String _val72;
-            for (int _i73 = 0; _i73 < _map70.size; ++_i73)
+            org.apache.thrift.protocol.TMap _map120 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); 
+            struct.success = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map120.size);
+            @org.apache.thrift.annotation.Nullable java.lang.String _key121;
+            @org.apache.thrift.annotation.Nullable java.lang.String _val122;
+            for (int _i123 = 0; _i123 < _map120.size; ++_i123)
             {
-              _key71 = iprot.readString();
-              _val72 = iprot.readString();
-              struct.success.put(_key71, _val72);
+              _key121 = iprot.readString();
+              _val122 = iprot.readString();
+              struct.success.put(_key121, _val122);
             }
           }
           struct.setSuccessIsSet(true);
         }
+        if (incoming.get(1)) {
+          struct.tope = new ThriftTableOperationException();
+          struct.tope.read(iprot);
+          struct.setTopeIsSet(true);
+        }
       }
     }
 
@@ -31762,25 +38864,25 @@ public class ClientService {
     }
   }
 
-  public static class getTableConfiguration_args implements org.apache.thrift.TBase<getTableConfiguration_args, getTableConfiguration_args._Fields>, java.io.Serializable, Cloneable, Comparable<getTableConfiguration_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getTableConfiguration_args");
+  public static class getNamespaceProperties_args implements org.apache.thrift.TBase<getNamespaceProperties_args, getNamespaceProperties_args._Fields>, java.io.Serializable, Cloneable, Comparable<getNamespaceProperties_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getNamespaceProperties_args");
 
     private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)1);
-    private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)3);
-    private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)2);
+    private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+    private static final org.apache.thrift.protocol.TField NS_FIELD_DESC = new org.apache.thrift.protocol.TField("ns", org.apache.thrift.protocol.TType.STRING, (short)3);
 
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getTableConfiguration_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getTableConfiguration_argsTupleSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getNamespaceProperties_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getNamespaceProperties_argsTupleSchemeFactory();
 
     public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
     public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
-    public @org.apache.thrift.annotation.Nullable java.lang.String tableName; // required
+    public @org.apache.thrift.annotation.Nullable java.lang.String ns; // 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 {
       TINFO((short)1, "tinfo"),
-      CREDENTIALS((short)3, "credentials"),
-      TABLE_NAME((short)2, "tableName");
+      CREDENTIALS((short)2, "credentials"),
+      NS((short)3, "ns");
 
       private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
 
@@ -31798,10 +38900,10 @@ public class ClientService {
         switch(fieldId) {
           case 1: // TINFO
             return TINFO;
-          case 3: // CREDENTIALS
+          case 2: // CREDENTIALS
             return CREDENTIALS;
-          case 2: // TABLE_NAME
-            return TABLE_NAME;
+          case 3: // NS
+            return NS;
           default:
             return null;
         }
@@ -31850,50 +38952,50 @@ public class ClientService {
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
       tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
-      tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+      tmpMap.put(_Fields.NS, new org.apache.thrift.meta_data.FieldMetaData("ns", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getTableConfiguration_args.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getNamespaceProperties_args.class, metaDataMap);
     }
 
-    public getTableConfiguration_args() {
+    public getNamespaceProperties_args() {
     }
 
-    public getTableConfiguration_args(
+    public getNamespaceProperties_args(
       org.apache.accumulo.core.trace.thrift.TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
-      java.lang.String tableName)
+      java.lang.String ns)
     {
       this();
       this.tinfo = tinfo;
       this.credentials = credentials;
-      this.tableName = tableName;
+      this.ns = ns;
     }
 
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public getTableConfiguration_args(getTableConfiguration_args other) {
+    public getNamespaceProperties_args(getNamespaceProperties_args other) {
       if (other.isSetTinfo()) {
         this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
       }
-      if (other.isSetTableName()) {
-        this.tableName = other.tableName;
+      if (other.isSetNs()) {
+        this.ns = other.ns;
       }
     }
 
-    public getTableConfiguration_args deepCopy() {
-      return new getTableConfiguration_args(this);
+    public getNamespaceProperties_args deepCopy() {
+      return new getNamespaceProperties_args(this);
     }
 
     @Override
     public void clear() {
       this.tinfo = null;
       this.credentials = null;
-      this.tableName = null;
+      this.ns = null;
     }
 
     @org.apache.thrift.annotation.Nullable
@@ -31901,7 +39003,7 @@ public class ClientService {
       return this.tinfo;
     }
 
-    public getTableConfiguration_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public getNamespaceProperties_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -31926,7 +39028,7 @@ public class ClientService {
       return this.credentials;
     }
 
-    public getTableConfiguration_args setCredentials(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) {
+    public getNamespaceProperties_args setCredentials(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) {
       this.credentials = credentials;
       return this;
     }
@@ -31947,27 +39049,27 @@ public class ClientService {
     }
 
     @org.apache.thrift.annotation.Nullable
-    public java.lang.String getTableName() {
-      return this.tableName;
+    public java.lang.String getNs() {
+      return this.ns;
     }
 
-    public getTableConfiguration_args setTableName(@org.apache.thrift.annotation.Nullable java.lang.String tableName) {
-      this.tableName = tableName;
+    public getNamespaceProperties_args setNs(@org.apache.thrift.annotation.Nullable java.lang.String ns) {
+      this.ns = ns;
       return this;
     }
 
-    public void unsetTableName() {
-      this.tableName = null;
+    public void unsetNs() {
+      this.ns = null;
     }
 
-    /** Returns true if field tableName is set (has been assigned a value) and false otherwise */
-    public boolean isSetTableName() {
-      return this.tableName != null;
+    /** Returns true if field ns is set (has been assigned a value) and false otherwise */
+    public boolean isSetNs() {
+      return this.ns != null;
     }
 
-    public void setTableNameIsSet(boolean value) {
+    public void setNsIsSet(boolean value) {
       if (!value) {
-        this.tableName = null;
+        this.ns = null;
       }
     }
 
@@ -31989,11 +39091,11 @@ public class ClientService {
         }
         break;
 
-      case TABLE_NAME:
+      case NS:
         if (value == null) {
-          unsetTableName();
+          unsetNs();
         } else {
-          setTableName((java.lang.String)value);
+          setNs((java.lang.String)value);
         }
         break;
 
@@ -32009,8 +39111,8 @@ public class ClientService {
       case CREDENTIALS:
         return getCredentials();
 
-      case TABLE_NAME:
-        return getTableName();
+      case NS:
+        return getNs();
 
       }
       throw new java.lang.IllegalStateException();
@@ -32027,20 +39129,20 @@ public class ClientService {
         return isSetTinfo();
       case CREDENTIALS:
         return isSetCredentials();
-      case TABLE_NAME:
-        return isSetTableName();
+      case NS:
+        return isSetNs();
       }
       throw new java.lang.IllegalStateException();
     }
 
     @Override
     public boolean equals(java.lang.Object that) {
-      if (that instanceof getTableConfiguration_args)
-        return this.equals((getTableConfiguration_args)that);
+      if (that instanceof getNamespaceProperties_args)
+        return this.equals((getNamespaceProperties_args)that);
       return false;
     }
 
-    public boolean equals(getTableConfiguration_args that) {
+    public boolean equals(getNamespaceProperties_args that) {
       if (that == null)
         return false;
       if (this == that)
@@ -32064,12 +39166,12 @@ public class ClientService {
           return false;
       }
 
-      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))
+      boolean this_present_ns = true && this.isSetNs();
+      boolean that_present_ns = true && that.isSetNs();
+      if (this_present_ns || that_present_ns) {
+        if (!(this_present_ns && that_present_ns))
           return false;
-        if (!this.tableName.equals(that.tableName))
+        if (!this.ns.equals(that.ns))
           return false;
       }
 
@@ -32088,15 +39190,15 @@ public class ClientService {
       if (isSetCredentials())
         hashCode = hashCode * 8191 + credentials.hashCode();
 
-      hashCode = hashCode * 8191 + ((isSetTableName()) ? 131071 : 524287);
-      if (isSetTableName())
-        hashCode = hashCode * 8191 + tableName.hashCode();
+      hashCode = hashCode * 8191 + ((isSetNs()) ? 131071 : 524287);
+      if (isSetNs())
+        hashCode = hashCode * 8191 + ns.hashCode();
 
       return hashCode;
     }
 
     @Override
-    public int compareTo(getTableConfiguration_args other) {
+    public int compareTo(getNamespaceProperties_args other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -32123,12 +39225,12 @@ public class ClientService {
           return lastComparison;
         }
       }
-      lastComparison = java.lang.Boolean.compare(isSetTableName(), other.isSetTableName());
+      lastComparison = java.lang.Boolean.compare(isSetNs(), other.isSetNs());
       if (lastComparison != 0) {
         return lastComparison;
       }
-      if (isSetTableName()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, other.tableName);
+      if (isSetNs()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.ns, other.ns);
         if (lastComparison != 0) {
           return lastComparison;
         }
@@ -32151,7 +39253,7 @@ public class ClientService {
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("getTableConfiguration_args(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("getNamespaceProperties_args(");
       boolean first = true;
 
       sb.append("tinfo:");
@@ -32170,11 +39272,11 @@ public class ClientService {
       }
       first = false;
       if (!first) sb.append(", ");
-      sb.append("tableName:");
-      if (this.tableName == null) {
+      sb.append("ns:");
+      if (this.ns == null) {
         sb.append("null");
       } else {
-        sb.append(this.tableName);
+        sb.append(this.ns);
       }
       first = false;
       sb.append(")");
@@ -32208,15 +39310,15 @@ public class ClientService {
       }
     }
 
-    private static class getTableConfiguration_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public getTableConfiguration_argsStandardScheme getScheme() {
-        return new getTableConfiguration_argsStandardScheme();
+    private static class getNamespaceProperties_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public getNamespaceProperties_argsStandardScheme getScheme() {
+        return new getNamespaceProperties_argsStandardScheme();
       }
     }
... 12720 lines suppressed ...