Merge branch '2.1'
diff --git a/LICENSE b/LICENSE
index 1c80cd5..22519c1 100644
--- a/LICENSE
+++ b/LICENSE
@@ -291,7 +291,8 @@
 
     Copyright (c) 2007-2014 IOLA and Ole Laursen
 
-    Available under the MIT License (see above and https://github.com/flot/flot/blob/master/LICENSE.txt)
+    Available under the MIT License
+    (see server/monitor/src/main/resources/org/apache/accumulo/monitor/resources/external/flot/LICENSE.txt)
 
     Flot bundles additional works:
 
diff --git a/assemble/bin/accumulo-cluster b/assemble/bin/accumulo-cluster
index d31d659..f525093 100755
--- a/assemble/bin/accumulo-cluster
+++ b/assemble/bin/accumulo-cluster
@@ -49,14 +49,6 @@
 
 function parse_config {
 
-  for file in slaves tservers monitor tracers gc managers masters; do
-    if [[ -f ${conf}/${file} ]]; then
-      echo "ERROR: A '${file}' file was found in ${conf}/"
-      echo "Accumulo now uses cluster host configuration information from 'cluster.yaml' and requires that the '${file}' file not be present to reduce confusion."
-      exit 1
-    fi
-  done
-
   if [[ ! -f ${conf}/cluster.yaml ]]; then
     echo "ERROR: A 'cluster.yaml' file was not found at ${conf}/cluster.yaml"
     echo "Please make sure it exists and is configured with the host information. Run 'accumulo-cluster create-config' to create an example configuration."
@@ -149,16 +141,16 @@
       # using the value of $host
       #
       if [[ $# -gt 3 ]]; then
-        ACCUMULO_SERVICE_INSTANCE="${ACCUMULO_SERVICE_INSTANCE}" "${bin}/accumulo-service" "$service" "$control_cmd" "-a" "$host" "${@:4}"
+        ACCUMULO_SERVICE_INSTANCE="${ACCUMULO_SERVICE_INSTANCE}" "${bin}/accumulo-service" "$service" "$control_cmd" "-o" "general.process.bind.addr=$host" "${@:4}"
       else
-        ACCUMULO_SERVICE_INSTANCE="${ACCUMULO_SERVICE_INSTANCE}" "${bin}/accumulo-service" "$service" "$control_cmd" "-a" "$host"
+        ACCUMULO_SERVICE_INSTANCE="${ACCUMULO_SERVICE_INSTANCE}" "${bin}/accumulo-service" "$service" "$control_cmd" "-o" "general.process.bind.addr=$host"
       fi
     else
       if [[ $# -gt 3 ]]; then
         EXTRA_ARGS="${*:4}"
-        $SSH "$host" "bash -c 'ACCUMULO_SERVICE_INSTANCE=${ACCUMULO_SERVICE_INSTANCE} ${bin}/accumulo-service \"$service\" \"$control_cmd\" \"-a\" \"$host\" $EXTRA_ARGS '"
+        $SSH "$host" "bash -c 'ACCUMULO_SERVICE_INSTANCE=${ACCUMULO_SERVICE_INSTANCE} ${bin}/accumulo-service \"$service\" \"$control_cmd\" \"-o\" \"general.process.bind.addr=$host\" $EXTRA_ARGS '"
       else
-        $SSH "$host" "bash -c 'ACCUMULO_SERVICE_INSTANCE=${ACCUMULO_SERVICE_INSTANCE} ${bin}/accumulo-service \"$service\" \"$control_cmd\" \"-a\" \"$host\"'"
+        $SSH "$host" "bash -c 'ACCUMULO_SERVICE_INSTANCE=${ACCUMULO_SERVICE_INSTANCE} ${bin}/accumulo-service \"$service\" \"$control_cmd\" \"-o\" \"general.process.bind.addr=$host\"'"
       fi
     fi
   done
@@ -204,7 +196,7 @@
   for group in $SSERVER_GROUPS; do
     G="SSERVER_HOSTS_${group}"
     for sserver in ${!G}; do
-      start_service "$sserver" sserver "-g" "$group"
+      start_service "$sserver" sserver "-o" "sserver.group=$group"
     done
   done
 
@@ -215,7 +207,7 @@
   for queue in $COMPACTION_QUEUES; do
     Q="COMPACTOR_HOSTS_${queue}"
     for compactor in ${!Q}; do
-      start_service "$compactor" compactor "-q" "$queue"
+      start_service "$compactor" compactor "-o" "compactor.queue=$queue"
     done
   done
 
@@ -341,7 +333,7 @@
   for group in $SSERVER_GROUPS; do
     G="SSERVER_HOSTS_${group}"
     for sserver in ${!G}; do
-      kill_service "$sserver" sserver "-g" "$group"
+      kill_service "$sserver" sserver "-o" "sserver.group=$group"
     done
   done
 
@@ -356,7 +348,7 @@
   for queue in $COMPACTION_QUEUES; do
     Q="COMPACTOR_HOSTS_${queue}"
     for compactor in ${!Q}; do
-      kill_service "$compactor" compactor "-q" "$queue"
+      kill_service "$compactor" compactor "-o" "compactor.queue=$queue"
     done
   done
 
@@ -397,7 +389,7 @@
     for group in $SSERVER_GROUPS; do
       G="SSERVER_HOSTS_${group}"
       for sserver in ${!G}; do
-        end_service $end_cmd "$sserver" sserver "-g" "$group"
+        end_service $end_cmd "$sserver" sserver "-o" "sserver.group=$group"
       done
     done
 
@@ -408,7 +400,7 @@
     for queue in $COMPACTION_QUEUES; do
       Q="COMPACTOR_HOSTS_${queue}"
       for compactor in ${!Q}; do
-        end_service $end_cmd "$compactor" compactor "-q" "$queue"
+        end_service $end_cmd "$compactor" compactor "-o" "compactor.queue=$queue"
       done
     done
 
@@ -445,13 +437,13 @@
       for group in $SSERVER_GROUPS; do
         G="SSERVER_HOSTS_${group}"
         for sserver in ${!G}; do
-          end_service $end_cmd "$sserver" sserver "-g" "$group"
+          end_service $end_cmd "$sserver" sserver "-o" "sserver.group=$group"
         done
       done
       for queue in $COMPACTION_QUEUES; do
         Q="COMPACTOR_HOSTS_${queue}"
         for compactor in ${!Q}; do
-          end_service $end_cmd "$host" compactor "-q" "$queue"
+          end_service $end_cmd "$host" compactor "-o" "compactor.queue=$queue"
         done
       done
     done
diff --git a/assemble/bin/accumulo-service b/assemble/bin/accumulo-service
index 552c2b2..c501876 100755
--- a/assemble/bin/accumulo-service
+++ b/assemble/bin/accumulo-service
@@ -26,7 +26,6 @@
   gc                     Accumulo garbage collector
   monitor                Accumulo monitor
   manager                Accumulo manager
-  master                 Accumulo master (Deprecated)
   tserver                Accumulo tserver
   compaction-coordinator Accumulo compaction coordinator (experimental)
   compactor              Accumulo compactor (experimental)
@@ -143,11 +142,6 @@
   fi
   service="$1"
 
-  if [[ $service == "master" ]]; then
-    echo "WARN : Use of 'master' service name is deprecated; use 'manager' instead."
-    service="manager"
-  fi
-
   pid_file="${ACCUMULO_PID_DIR}/accumulo-${service}${ACCUMULO_SERVICE_INSTANCE}.pid"
   case "$service" in
     gc | manager | monitor | tserver | compaction-coordinator | compactor | sserver)
diff --git a/assemble/conf/accumulo-env.sh b/assemble/conf/accumulo-env.sh
index bcde690..9e4acc1 100644
--- a/assemble/conf/accumulo-env.sh
+++ b/assemble/conf/accumulo-env.sh
@@ -90,7 +90,7 @@
 # cmd is set by calling script that sources this env file
 #shellcheck disable=SC2154
 case "$cmd" in
-  manager | master) JAVA_OPTS=('-Xmx512m' '-Xms512m' "${JAVA_OPTS[@]}") ;;
+  manager) JAVA_OPTS=('-Xmx512m' '-Xms512m' "${JAVA_OPTS[@]}") ;;
   monitor) JAVA_OPTS=('-Xmx256m' '-Xms256m' "${JAVA_OPTS[@]}") ;;
   gc) JAVA_OPTS=('-Xmx256m' '-Xms256m' "${JAVA_OPTS[@]}") ;;
   tserver) JAVA_OPTS=('-Xmx768m' '-Xms768m' "${JAVA_OPTS[@]}") ;;
@@ -118,7 +118,7 @@
 #JAVA_OPTS=('-javaagent:path/to/opentelemetry-javaagent-all.jar' "${JAVA_OPTS[@]}")
 
 case "$cmd" in
-  monitor | gc | manager | master | tserver | compaction-coordinator | compactor | sserver)
+  monitor | gc | manager | tserver | compaction-coordinator | compactor | sserver)
     JAVA_OPTS=('-Dlog4j.configurationFile=log4j2-service.properties' "${JAVA_OPTS[@]}")
     ;;
   *)
diff --git a/assemble/pom.xml b/assemble/pom.xml
index d24a608..4b7fe16 100644
--- a/assemble/pom.xml
+++ b/assemble/pom.xml
@@ -24,7 +24,7 @@
   <parent>
     <groupId>org.apache.accumulo</groupId>
     <artifactId>accumulo-project</artifactId>
-    <version>2.1.3-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
   </parent>
   <artifactId>accumulo</artifactId>
   <packaging>pom</packaging>
@@ -92,11 +92,6 @@
       <optional>true</optional>
     </dependency>
     <dependency>
-      <groupId>com.google.protobuf</groupId>
-      <artifactId>protobuf-java</artifactId>
-      <optional>true</optional>
-    </dependency>
-    <dependency>
       <groupId>com.lmax</groupId>
       <artifactId>disruptor</artifactId>
       <optional>true</optional>
@@ -188,6 +183,11 @@
     </dependency>
     <dependency>
       <groupId>org.apache.accumulo</groupId>
+      <artifactId>accumulo-access</artifactId>
+      <optional>true</optional>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.accumulo</groupId>
       <artifactId>accumulo-compaction-coordinator</artifactId>
       <optional>true</optional>
     </dependency>
@@ -278,11 +278,6 @@
       <optional>true</optional>
     </dependency>
     <dependency>
-      <groupId>org.apache.commons</groupId>
-      <artifactId>commons-vfs2</artifactId>
-      <optional>true</optional>
-    </dependency>
-    <dependency>
       <groupId>org.apache.datasketches</groupId>
       <artifactId>datasketches-java</artifactId>
       <optional>true</optional>
diff --git a/assemble/src/main/resources/LICENSE b/assemble/src/main/resources/LICENSE
index e01f6ae..9bf302b 100644
--- a/assemble/src/main/resources/LICENSE
+++ b/assemble/src/main/resources/LICENSE
@@ -290,7 +290,8 @@
 
     Copyright (c) 2007-2014 IOLA and Ole Laursen
 
-    Available under the MIT License (see above and https://github.com/flot/flot/blob/master/LICENSE.txt)
+    Available under the MIT License
+    (see org/apache/accumulo/monitor/resources/external/flot/LICENSE.txt in the accumulo-monitor jar)
 
     Flot bundles additional works:
 
diff --git a/core/pom.xml b/core/pom.xml
index 9822afe..fcaecd8 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -24,7 +24,7 @@
   <parent>
     <groupId>org.apache.accumulo</groupId>
     <artifactId>accumulo-project</artifactId>
-    <version>2.1.3-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
   </parent>
   <artifactId>accumulo-core</artifactId>
   <name>Apache Accumulo Core</name>
@@ -62,10 +62,6 @@
       </exclusions>
     </dependency>
     <dependency>
-      <groupId>com.google.protobuf</groupId>
-      <artifactId>protobuf-java</artifactId>
-    </dependency>
-    <dependency>
       <groupId>commons-io</groupId>
       <artifactId>commons-io</artifactId>
     </dependency>
@@ -87,6 +83,10 @@
     </dependency>
     <dependency>
       <groupId>org.apache.accumulo</groupId>
+      <artifactId>accumulo-access</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.accumulo</groupId>
       <artifactId>accumulo-start</artifactId>
     </dependency>
     <dependency>
@@ -209,7 +209,6 @@
                   <exclude>src/main/java/org/apache/accumulo/core/bloomfilter/*.java</exclude>
                   <exclude>src/main/java/org/apache/accumulo/core/util/HostAndPort.java</exclude>
                   <exclude>src/test/resources/*.jceks</exclude>
-                  <exclude>src/test/resources/org/apache/accumulo/core/file/rfile/*.rf</exclude>
                 </excludes>
               </licenseSet>
             </licenseSets>
@@ -222,7 +221,6 @@
             <excludes>
               <exclude>src/main/java/org/apache/accumulo/core/bloomfilter/*.java</exclude>
               <exclude>src/test/resources/*.jceks</exclude>
-              <exclude>src/test/resources/org/apache/accumulo/core/file/rfile/*.rf</exclude>
             </excludes>
           </configuration>
         </plugin>
@@ -274,6 +272,8 @@
                 <allow>javax[.]security[.]auth[.]DestroyFailedException</allow>
                 <!-- allow questionable Hadoop exceptions for mapreduce -->
                 <allow>org[.]apache[.]hadoop[.]mapred[.](FileAlreadyExistsException|InvalidJobConfException)</allow>
+                <!-- allow the following types from the visibility API -->
+                <allow>org[.]apache[.]accumulo[.]access[.].*</allow>
               </allows>
             </configuration>
           </execution>
@@ -368,7 +368,7 @@
               <classpathScope>test</classpathScope>
               <arguments>
                 <argument>--generate-markdown</argument>
-                <argument>${project.build.directory}/generated-docs/server-properties.md</argument>
+                <argument>${project.build.directory}/generated-docs/server-properties3.md</argument>
               </arguments>
             </configuration>
           </execution>
@@ -383,7 +383,7 @@
               <classpathScope>test</classpathScope>
               <arguments>
                 <argument>--generate-markdown</argument>
-                <argument>${project.build.directory}/generated-docs/client-properties.md</argument>
+                <argument>${project.build.directory}/generated-docs/client-properties3.md</argument>
               </arguments>
             </configuration>
           </execution>
diff --git a/core/src/main/java/org/apache/accumulo/core/Constants.java b/core/src/main/java/org/apache/accumulo/core/Constants.java
index c31e205..411af75 100644
--- a/core/src/main/java/org/apache/accumulo/core/Constants.java
+++ b/core/src/main/java/org/apache/accumulo/core/Constants.java
@@ -35,6 +35,7 @@
   // Zookeeper locations
   public static final String ZROOT = "/accumulo";
   public static final String ZINSTANCES = "/instances";
+  public static final String ZUSERS = "/users";
 
   public static final String ZTABLES = "/tables";
   public static final byte[] ZTABLES_INITIAL_ID = {'0'};
@@ -52,7 +53,6 @@
   public static final String ZMANAGERS = "/managers";
   public static final String ZMANAGER_LOCK = ZMANAGERS + "/lock";
   public static final String ZMANAGER_GOAL_STATE = ZMANAGERS + "/goal_state";
-  public static final String ZMANAGER_REPLICATION_COORDINATOR_ADDR = ZMANAGERS + "/repl_coord_addr";
   public static final String ZMANAGER_TICK = ZMANAGERS + "/tick";
 
   public static final String ZGC = "/gc";
@@ -62,8 +62,6 @@
   public static final String ZMONITOR_LOCK = ZMONITOR + "/lock";
   public static final String ZMONITOR_HTTP_ADDR = ZMONITOR + "/http_addr";
 
-  // used by < 2.1 table and namespace configurations
-  public static final String ZCONF_LEGACY = "/conf";
   public static final String ZCONFIG = "/config";
 
   public static final String ZTSERVERS = "/tservers";
@@ -78,8 +76,6 @@
   public static final String ZDEAD = "/dead";
   public static final String ZDEADTSERVERS = ZDEAD + "/tservers";
 
-  public static final String ZTRACERS = "/tracers";
-
   public static final String ZPROBLEMS = "/problems";
 
   public static final String BULK_ARBITRATOR_TYPE = "bulkTx";
@@ -88,9 +84,6 @@
 
   public static final String ZNEXT_FILE = "/next_file";
 
-  // TODO: Remove when Property.TSERV_WORKQ_THREADS is removed
-  public static final String ZBULK_FAILED_COPYQ = "/bulk_failed_copyq";
-
   public static final String ZHDFS_RESERVATIONS = "/hdfs_reservations";
   public static final String ZRECOVERY = "/recovery";
 
@@ -117,7 +110,6 @@
 
   public static final int MAX_DATA_TO_PRINT = 64;
   public static final String CORE_PACKAGE_NAME = "org.apache.accumulo.core";
-  public static final String MAPFILE_EXTENSION = "map";
   public static final String GENERATED_TABLET_DIRECTORY_PREFIX = "t-";
 
   public static final String EXPORT_METADATA_FILE = "metadata.bin";
diff --git a/core/src/main/java/org/apache/accumulo/core/classloader/ClassLoaderUtil.java b/core/src/main/java/org/apache/accumulo/core/classloader/ClassLoaderUtil.java
index 4928767..dc84a63 100644
--- a/core/src/main/java/org/apache/accumulo/core/classloader/ClassLoaderUtil.java
+++ b/core/src/main/java/org/apache/accumulo/core/classloader/ClassLoaderUtil.java
@@ -45,7 +45,7 @@
         // load the default implementation
         LOG.info("Using default {}, which is subject to change in a future release",
             ContextClassLoaderFactory.class.getName());
-        FACTORY = new DefaultContextClassLoaderFactory(conf);
+        FACTORY = new URLContextClassLoaderFactory();
       } else {
         // load user's selected implementation and provide it with the service environment
         try {
@@ -73,12 +73,15 @@
     FACTORY = null;
   }
 
-  @SuppressWarnings("deprecation")
+  public static ClassLoader getClassLoader() {
+    return getClassLoader(null);
+  }
+
   public static ClassLoader getClassLoader(String context) {
     if (context != null && !context.isEmpty()) {
       return FACTORY.getClassLoader(context);
     } else {
-      return org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader.getClassLoader();
+      return ClassLoader.getSystemClassLoader();
     }
   }
 
@@ -114,9 +117,8 @@
   /**
    * Retrieve the classloader context from a table's configuration.
    */
-  @SuppressWarnings("removal")
   public static String tableContext(AccumuloConfiguration conf) {
-    return conf.get(conf.resolve(Property.TABLE_CLASSLOADER_CONTEXT, Property.TABLE_CLASSPATH));
+    return conf.get(Property.TABLE_CLASSLOADER_CONTEXT);
   }
 
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/classloader/DefaultContextClassLoaderFactory.java b/core/src/main/java/org/apache/accumulo/core/classloader/DefaultContextClassLoaderFactory.java
deleted file mode 100644
index 152c2b9..0000000
--- a/core/src/main/java/org/apache/accumulo/core/classloader/DefaultContextClassLoaderFactory.java
+++ /dev/null
@@ -1,101 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.classloader;
-
-import static java.util.concurrent.TimeUnit.MINUTES;
-
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ScheduledFuture;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.function.Supplier;
-import java.util.stream.Collectors;
-
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.spi.common.ContextClassLoaderFactory;
-import org.apache.accumulo.core.util.threads.ThreadPools;
-import org.apache.accumulo.core.util.threads.Threads;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * The default implementation of ContextClassLoaderFactory. This implementation is subject to change
- * over time. It currently implements the legacy context class loading behavior based on Accumulo's
- * custom class loaders and commons-vfs2. In future, it may simply return the system class loader
- * for all requested contexts. This class is used internally to Accumulo only, and should not be
- * used by users in their configuration.
- */
-public class DefaultContextClassLoaderFactory implements ContextClassLoaderFactory {
-
-  private static final AtomicBoolean isInstantiated = new AtomicBoolean(false);
-  private static final Logger LOG = LoggerFactory.getLogger(DefaultContextClassLoaderFactory.class);
-  private static final String className = DefaultContextClassLoaderFactory.class.getName();
-
-  @SuppressWarnings("removal")
-  private static final Property VFS_CONTEXT_CLASSPATH_PROPERTY =
-      Property.VFS_CONTEXT_CLASSPATH_PROPERTY;
-
-  public DefaultContextClassLoaderFactory(final AccumuloConfiguration accConf) {
-    if (!isInstantiated.compareAndSet(false, true)) {
-      throw new IllegalStateException("Can only instantiate " + className + " once");
-    }
-    Supplier<Map<String,String>> contextConfigSupplier =
-        () -> accConf.getAllPropertiesWithPrefix(VFS_CONTEXT_CLASSPATH_PROPERTY);
-    setContextConfig(contextConfigSupplier);
-    LOG.debug("ContextManager configuration set");
-    startCleanupThread(accConf, contextConfigSupplier);
-  }
-
-  @SuppressWarnings("deprecation")
-  private static void setContextConfig(Supplier<Map<String,String>> contextConfigSupplier) {
-    org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader
-        .setContextConfig(contextConfigSupplier);
-  }
-
-  private static void startCleanupThread(final AccumuloConfiguration conf,
-      final Supplier<Map<String,String>> contextConfigSupplier) {
-    ScheduledFuture<?> future = ThreadPools.getClientThreadPools((t, e) -> {
-      LOG.error("context classloader cleanup thread has failed.", e);
-    }).createGeneralScheduledExecutorService(conf)
-        .scheduleWithFixedDelay(Threads.createNamedRunnable(className + "-cleanup", () -> {
-          LOG.trace("{}-cleanup thread, properties: {}", className, conf);
-          Set<String> contextsInUse = contextConfigSupplier.get().keySet().stream()
-              .map(p -> p.substring(VFS_CONTEXT_CLASSPATH_PROPERTY.getKey().length()))
-              .collect(Collectors.toSet());
-          LOG.trace("{}-cleanup thread, contexts in use: {}", className, contextsInUse);
-          removeUnusedContexts(contextsInUse);
-        }), 1, 1, MINUTES);
-    ThreadPools.watchNonCriticalScheduledTask(future);
-    LOG.debug("Context cleanup timer started at 60s intervals");
-  }
-
-  @SuppressWarnings("deprecation")
-  private static void removeUnusedContexts(Set<String> contextsInUse) {
-    org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader
-        .removeUnusedContexts(contextsInUse);
-  }
-
-  @SuppressWarnings("deprecation")
-  @Override
-  public ClassLoader getClassLoader(String contextName) {
-    return org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader
-        .getContextClassLoader(contextName);
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/classloader/URLContextClassLoaderFactory.java b/core/src/main/java/org/apache/accumulo/core/classloader/URLContextClassLoaderFactory.java
new file mode 100644
index 0000000..26b06e3
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/classloader/URLContextClassLoaderFactory.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.classloader;
+
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.util.Arrays;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.accumulo.core.spi.common.ContextClassLoaderFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+
+/**
+ * The default implementation of ContextClassLoaderFactory. This classloader returns a
+ * URLClassLoader based on the given context value which is a CSV list of URLs. For example,
+ * file://path/one/jar1.jar,file://path/two/jar2.jar
+ */
+public class URLContextClassLoaderFactory implements ContextClassLoaderFactory {
+
+  private static final AtomicBoolean isInstantiated = new AtomicBoolean(false);
+  private static final Logger LOG = LoggerFactory.getLogger(URLContextClassLoaderFactory.class);
+  private static final String className = URLContextClassLoaderFactory.class.getName();
+
+  // Cache the class loaders for re-use
+  // WeakReferences are used so that the class loaders can be cleaned up when no longer needed
+  // Classes that are loaded contain a reference to the class loader used to load them
+  // so the class loader will be garbage collected when no more classes are loaded that reference it
+  private final Cache<String,URLClassLoader> classloaders =
+      Caffeine.newBuilder().weakValues().build();
+
+  public URLContextClassLoaderFactory() {
+    if (!isInstantiated.compareAndSet(false, true)) {
+      throw new IllegalStateException("Can only instantiate " + className + " once");
+    }
+  }
+
+  @Override
+  public ClassLoader getClassLoader(String context) {
+    if (context == null) {
+      throw new IllegalArgumentException("Unknown context");
+    }
+
+    return classloaders.get(context, k -> {
+      LOG.debug("Creating URLClassLoader for context, uris: {}", context);
+      return new URLClassLoader(Arrays.stream(context.split(",")).map(url -> {
+        try {
+          return new URL(url);
+        } catch (MalformedURLException e) {
+          throw new RuntimeException(e);
+        }
+      }).toArray(URL[]::new), ClassLoader.getSystemClassLoader());
+    });
+  }
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/AccumuloClient.java b/core/src/main/java/org/apache/accumulo/core/client/AccumuloClient.java
index 25e6ab3..5b82d0e 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/AccumuloClient.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/AccumuloClient.java
@@ -51,11 +51,11 @@
  * </pre>
  *
  * <p>
- * If migrating code from Connector to AccumuloClient an important difference to consider is that
- * AccumuloClient is closable and Connector is not. Connector uses static resources and therefore
- * creating them is cheap. AccumuloClient attempts to clean up resources on close, so constantly
- * creating them could perform worse than Connector. Therefore, it would be better to create an
- * AccumuloClient and pass it around.
+ * An important difference with the legacy Connector to consider is that Connector reused global
+ * static resources. AccumuloClient, however, attempts to clean up its resources on close. So,
+ * creating many AccumuloClient objects will perform worse than creating many Connectors did.
+ * Therefore, it is suggested to reuse AccumuloClient instances where possible, rather than create
+ * many of them.
  *
  * <p>
  * AccumuloClient objects are intended to be thread-safe, and can be used by multiple threads.
@@ -292,14 +292,6 @@
   InstanceOperations instanceOperations();
 
   /**
-   * Retrieves a ReplicationOperations object to manage replication configuration.
-   *
-   * @return an object to modify replication configuration
-   */
-  @Deprecated(since = "2.1.0")
-  org.apache.accumulo.core.client.admin.ReplicationOperations replicationOperations();
-
-  /**
    * @return All {@link Properties} used to create client except 'auth.token'
    */
   Properties properties();
diff --git a/core/src/main/java/org/apache/accumulo/core/client/BatchWriter.java b/core/src/main/java/org/apache/accumulo/core/client/BatchWriter.java
index 68dbb5f..2bf7fcd 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/BatchWriter.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/BatchWriter.java
@@ -23,7 +23,7 @@
 /**
  * Send Mutations to a single Table in Accumulo.
  * <p>
- * When the user uses a Connector to create a BatchWriter, they specify how much memory and how many
+ * When the user uses a client to create a BatchWriter, they specify how much memory and how many
  * threads it should use. As the user adds mutations to the batch writer, it buffers them. Once the
  * buffered mutations have used half of the user specified buffer, the mutations are dumped into the
  * background to be written by a thread pool. If the user specified memory completely fills up, then
diff --git a/core/src/main/java/org/apache/accumulo/core/client/BatchWriterConfig.java b/core/src/main/java/org/apache/accumulo/core/client/BatchWriterConfig.java
index c570934..d3a8f7d 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/BatchWriterConfig.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/BatchWriterConfig.java
@@ -122,7 +122,6 @@
     } else {
       // make small, positive values that truncate to 0 when converted use the minimum millis
       // instead
-
       this.maxLatency = Math.max(1, timeUnit.toMillis(maxLatency));
     }
     return this;
@@ -157,7 +156,6 @@
     } else {
       // make small, positive values that truncate to 0 when converted use the minimum millis
       // instead
-
       this.timeout = Math.max(1, timeUnit.toMillis(timeout));
     }
     return this;
diff --git a/core/src/main/java/org/apache/accumulo/core/client/ClientConfiguration.java b/core/src/main/java/org/apache/accumulo/core/client/ClientConfiguration.java
deleted file mode 100644
index 89bab19..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/ClientConfiguration.java
+++ /dev/null
@@ -1,547 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.client;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static java.nio.charset.StandardCharsets.UTF_8;
-
-import java.io.File;
-import java.io.FileReader;
-import java.io.IOException;
-import java.io.StringReader;
-import java.io.StringWriter;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.UUID;
-
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.conf.PropertyType;
-import org.apache.commons.configuration2.CompositeConfiguration;
-import org.apache.commons.configuration2.Configuration;
-import org.apache.commons.configuration2.MapConfiguration;
-import org.apache.commons.configuration2.PropertiesConfiguration;
-import org.apache.commons.configuration2.ex.ConfigurationException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
-/**
- * Contains a list of property keys recognized by the Accumulo client and convenience methods for
- * setting them.
- *
- * @since 1.6.0
- * @deprecated since 2.0.0, replaced by {@link Accumulo#newClient()}
- */
-@Deprecated(since = "2.0.0")
-public class ClientConfiguration {
-  private static final Logger log = LoggerFactory.getLogger(ClientConfiguration.class);
-
-  public static final String USER_ACCUMULO_DIR_NAME = ".accumulo";
-  public static final String USER_CONF_FILENAME = "config";
-  public static final String GLOBAL_CONF_FILENAME = "client.conf";
-
-  private final CompositeConfiguration compositeConfig;
-
-  public enum ClientProperty {
-    // SSL
-    RPC_SSL_TRUSTSTORE_PATH(Property.RPC_SSL_TRUSTSTORE_PATH),
-    RPC_SSL_TRUSTSTORE_PASSWORD(Property.RPC_SSL_TRUSTSTORE_PASSWORD),
-    RPC_SSL_TRUSTSTORE_TYPE(Property.RPC_SSL_TRUSTSTORE_TYPE),
-    RPC_SSL_KEYSTORE_PATH(Property.RPC_SSL_KEYSTORE_PATH),
-    RPC_SSL_KEYSTORE_PASSWORD(Property.RPC_SSL_KEYSTORE_PASSWORD),
-    RPC_SSL_KEYSTORE_TYPE(Property.RPC_SSL_KEYSTORE_TYPE),
-    RPC_USE_JSSE(Property.RPC_USE_JSSE),
-    GENERAL_SECURITY_CREDENTIAL_PROVIDER_PATHS(Property.GENERAL_SECURITY_CREDENTIAL_PROVIDER_PATHS),
-    INSTANCE_RPC_SSL_CLIENT_AUTH(Property.INSTANCE_RPC_SSL_CLIENT_AUTH),
-    INSTANCE_RPC_SSL_ENABLED(Property.INSTANCE_RPC_SSL_ENABLED),
-
-    // ZooKeeper
-    INSTANCE_ZK_HOST(Property.INSTANCE_ZK_HOST),
-    INSTANCE_ZK_TIMEOUT(Property.INSTANCE_ZK_TIMEOUT),
-
-    // Instance information
-    INSTANCE_NAME("instance.name", null, PropertyType.STRING,
-        "Name of Accumulo instance to connect to"),
-    INSTANCE_ID("instance.id", null, PropertyType.STRING,
-        "UUID of Accumulo instance to connect to"),
-
-    // Tracing
-    @Deprecated(since = "2.1.0")
-    TRACE_SPAN_RECEIVERS(Property.TRACE_SPAN_RECEIVERS),
-    @Deprecated(since = "2.1.0")
-    TRACE_SPAN_RECEIVER_PREFIX(Property.TRACE_SPAN_RECEIVER_PREFIX),
-    @Deprecated(since = "2.1.0")
-    TRACE_ZK_PATH(Property.TRACE_ZK_PATH),
-
-    // SASL / GSSAPI(Kerberos)
-    /**
-     * @since 1.7.0
-     */
-    INSTANCE_RPC_SASL_ENABLED(Property.INSTANCE_RPC_SASL_ENABLED),
-    /**
-     * @since 1.7.0
-     */
-    RPC_SASL_QOP(Property.RPC_SASL_QOP),
-    /**
-     * @since 1.7.0
-     */
-    KERBEROS_SERVER_PRIMARY("kerberos.server.primary", "accumulo", PropertyType.STRING,
-        "The first component of the Kerberos principal, the 'primary', "
-            + "that Accumulo servers use to login");
-
-    private String key;
-    private String defaultValue;
-    private PropertyType type;
-    private String description;
-
-    private ClientProperty(Property prop) {
-      this(prop.getKey(), prop.getDefaultValue(), prop.getType(), prop.getDescription());
-    }
-
-    private ClientProperty(String key, String defaultValue, PropertyType type, String description) {
-      this.key = key;
-      this.defaultValue = defaultValue;
-      this.type = type;
-      this.description = description;
-    }
-
-    public String getKey() {
-      return key;
-    }
-
-    public String getDefaultValue() {
-      return defaultValue;
-    }
-
-    private PropertyType getType() {
-      return type;
-    }
-
-    public String getDescription() {
-      return description;
-    }
-
-    public static ClientProperty getPropertyByKey(String key) {
-      for (ClientProperty prop : ClientProperty.values()) {
-        if (prop.getKey().equals(key)) {
-          return prop;
-        }
-      }
-      return null;
-    }
-  }
-
-  private ClientConfiguration(List<? extends Configuration> configs) {
-    compositeConfig = new CompositeConfiguration(configs);
-  }
-
-  /**
-   * Attempts to load a configuration file from the system using the default search paths. Uses the
-   * <em>ACCUMULO_CLIENT_CONF_PATH</em> environment variable, split on <em>File.pathSeparator</em>,
-   * for a list of target files.
-   * <p>
-   * If <em>ACCUMULO_CLIENT_CONF_PATH</em> is not set, uses the following in this order:
-   * <ul>
-   * <li>~/.accumulo/config
-   * <li><em>$ACCUMULO_CONF_DIR</em>/client.conf, if <em>$ACCUMULO_CONF_DIR</em> is defined.
-   * <li>/etc/accumulo/client.conf
-   * <li>/etc/accumulo/conf/client.conf
-   * </ul>
-   * <p>
-   * A client configuration will then be read from each location using
-   * <em>PropertiesConfiguration</em> to construct a configuration. That means the latest item will
-   * be the one in the configuration.
-   *
-   * @see PropertiesConfiguration
-   * @see File#pathSeparator
-   */
-  public static ClientConfiguration loadDefault() {
-    return loadFromSearchPath(getDefaultSearchPath());
-  }
-
-  /**
-   * Initializes an empty configuration object to be further configured with other methods on the
-   * class.
-   *
-   * @since 1.9.0
-   */
-  public static ClientConfiguration create() {
-    return new ClientConfiguration(Collections.emptyList());
-  }
-
-  /**
-   * Initializes a configuration object from the contents of a configuration file. Currently
-   * supports Java "properties" files. The returned object can be further configured with subsequent
-   * calls to other methods on this class.
-   *
-   * @param file the path to the configuration file
-   * @since 1.9.0
-   */
-  public static ClientConfiguration fromFile(File file) {
-    var config = new PropertiesConfiguration();
-    try (var reader = new FileReader(file, UTF_8)) {
-      config.read(reader);
-    } catch (ConfigurationException | IOException e) {
-      throw new IllegalArgumentException("Bad configuration file: " + file, e);
-    }
-    return new ClientConfiguration(Collections.singletonList(config));
-  }
-
-  /**
-   * Initializes a configuration object from the contents of a map. The returned object can be
-   * further configured with subsequent calls to other methods on this class.
-   *
-   * @param properties a map containing the configuration properties to use
-   * @since 1.9.0
-   */
-  public static ClientConfiguration fromMap(Map<String,String> properties) {
-    MapConfiguration mapConf = new MapConfiguration(properties);
-    return new ClientConfiguration(Collections.singletonList(mapConf));
-  }
-
-  @SuppressFBWarnings(value = "PATH_TRAVERSAL_IN",
-      justification = "process runs in same security context as user who provided path")
-  private static ClientConfiguration loadFromSearchPath(List<String> paths) {
-    List<Configuration> configs = new LinkedList<>();
-    for (String path : paths) {
-      File conf = new File(path);
-      if (conf.isFile() && conf.canRead()) {
-        var config = new PropertiesConfiguration();
-        try (var reader = new FileReader(conf, UTF_8)) {
-          config.read(reader);
-        } catch (ConfigurationException | IOException e) {
-          throw new IllegalStateException("Error loading client configuration file " + conf, e);
-        }
-        configs.add(config);
-        log.info("Loaded client configuration file {}", conf);
-      }
-    }
-    // We couldn't find the client configuration anywhere
-    if (configs.isEmpty()) {
-      log.debug(
-          "Found no client.conf in default paths. Using default client configuration values.");
-    }
-    return new ClientConfiguration(configs);
-  }
-
-  public static ClientConfiguration deserialize(String serializedConfig) {
-    var propConfig = new PropertiesConfiguration();
-    try {
-      propConfig.read(new StringReader(serializedConfig));
-    } catch (ConfigurationException | IOException e) {
-      throw new IllegalArgumentException(
-          "Error deserializing client configuration: " + serializedConfig, e);
-    }
-    return new ClientConfiguration(Collections.singletonList(propConfig));
-  }
-
-  /**
-   * Muck the value of {@code clientConfPath} if it points to a directory by appending
-   * {@code client.conf} to the end of the file path. This is a no-op if the value is not a
-   * directory on the filesystem.
-   *
-   * @param clientConfPath The value of ACCUMULO_CLIENT_CONF_PATH.
-   */
-  @SuppressFBWarnings(value = "PATH_TRAVERSAL_IN",
-      justification = "process runs in same security context as user who provided path")
-  static String getClientConfPath(String clientConfPath) {
-    if (clientConfPath == null) {
-      return null;
-    }
-    File filePath = new File(clientConfPath);
-    // If clientConfPath is a directory, tack on the default client.conf file name.
-    if (filePath.exists() && filePath.isDirectory()) {
-      return new File(filePath, "client.conf").toString();
-    }
-    return clientConfPath;
-  }
-
-  private static List<String> getDefaultSearchPath() {
-    String clientConfSearchPath = getClientConfPath(System.getenv("ACCUMULO_CLIENT_CONF_PATH"));
-    List<String> clientConfPaths;
-    if (clientConfSearchPath != null) {
-      clientConfPaths = Arrays.asList(clientConfSearchPath.split(File.pathSeparator));
-    } else {
-      // if $ACCUMULO_CLIENT_CONF_PATH env isn't set, priority from top to bottom is:
-      // ~/.accumulo/config
-      // $ACCUMULO_CONF_DIR/client.conf
-      // /etc/accumulo/client.conf
-      // /etc/accumulo/conf/client.conf
-      clientConfPaths = new LinkedList<>();
-      clientConfPaths.add(System.getProperty("user.home") + File.separator + USER_ACCUMULO_DIR_NAME
-          + File.separator + USER_CONF_FILENAME);
-      if (System.getenv("ACCUMULO_CONF_DIR") != null) {
-        clientConfPaths
-            .add(System.getenv("ACCUMULO_CONF_DIR") + File.separator + GLOBAL_CONF_FILENAME);
-      }
-      clientConfPaths.add("/etc/accumulo/" + GLOBAL_CONF_FILENAME);
-      clientConfPaths.add("/etc/accumulo/conf/" + GLOBAL_CONF_FILENAME);
-    }
-    return clientConfPaths;
-  }
-
-  public String serialize() {
-    var propConfig = new PropertiesConfiguration();
-    propConfig.copy(compositeConfig);
-    StringWriter writer = new StringWriter();
-    try {
-      propConfig.write(writer);
-    } catch (ConfigurationException | IOException e) {
-      // this should never happen
-      throw new IllegalStateException(e);
-    }
-    return writer.toString();
-  }
-
-  /**
-   * Returns the value for prop, the default value if not present.
-   *
-   */
-  public String get(ClientProperty prop) {
-    if (compositeConfig.containsKey(prop.getKey())) {
-      return compositeConfig.getString(prop.getKey());
-    } else {
-      return prop.getDefaultValue();
-    }
-  }
-
-  private void checkType(ClientProperty property, PropertyType type) {
-    if (!property.getType().equals(type)) {
-      String msg = "Configuration method intended for type " + type + " called with a "
-          + property.getType() + " argument (" + property.getKey() + ")";
-      throw new IllegalArgumentException(msg);
-    }
-  }
-
-  /**
-   * Gets all properties under the given prefix in this configuration.
-   *
-   * @param property prefix property, must be of type PropertyType.PREFIX
-   * @return a map of property keys to values
-   * @throws IllegalArgumentException if property is not a prefix
-   */
-  public Map<String,String> getAllPropertiesWithPrefix(ClientProperty property) {
-    checkType(property, PropertyType.PREFIX);
-
-    Map<String,String> propMap = new HashMap<>();
-    String prefix = property.getKey();
-    if (prefix.endsWith(".")) {
-      prefix = prefix.substring(0, prefix.length() - 1);
-    }
-    Iterator<?> iter = compositeConfig.getKeys(prefix);
-    while (iter.hasNext()) {
-      String p = (String) iter.next();
-      propMap.put(p, compositeConfig.getString(p));
-    }
-    return propMap;
-  }
-
-  /**
-   * Sets the value of property to value
-   *
-   */
-  public void setProperty(ClientProperty prop, String value) {
-    with(prop, value);
-  }
-
-  /**
-   * Same as {@link #setProperty(ClientProperty, String)} but returns the ClientConfiguration for
-   * chaining purposes
-   */
-  public ClientConfiguration with(ClientProperty prop, String value) {
-    return with(prop.getKey(), value);
-  }
-
-  /**
-   * Sets the value of property to value
-   *
-   * @since 1.9.0
-   */
-  public void setProperty(String prop, String value) {
-    with(prop, value);
-  }
-
-  /**
-   * Same as {@link #setProperty(String, String)} but returns the ClientConfiguration for chaining
-   * purposes
-   *
-   * @since 1.9.0
-   */
-  public ClientConfiguration with(String prop, String value) {
-    compositeConfig.setProperty(prop, value);
-    return this;
-  }
-
-  /**
-   * Same as {@link #with(ClientProperty, String)} for ClientProperty.INSTANCE_NAME
-   *
-   */
-  public ClientConfiguration withInstance(String instanceName) {
-    checkArgument(instanceName != null, "instanceName is null");
-    return with(ClientProperty.INSTANCE_NAME, instanceName);
-  }
-
-  /**
-   * Same as {@link #with(ClientProperty, String)} for ClientProperty.INSTANCE_ID
-   *
-   */
-  public ClientConfiguration withInstance(UUID instanceId) {
-    checkArgument(instanceId != null, "instanceId is null");
-    return with(ClientProperty.INSTANCE_ID, instanceId.toString());
-  }
-
-  /**
-   * Same as {@link #with(ClientProperty, String)} for ClientProperty.INSTANCE_ZK_HOST
-   *
-   */
-  public ClientConfiguration withZkHosts(String zooKeepers) {
-    checkArgument(zooKeepers != null, "zooKeepers is null");
-    return with(ClientProperty.INSTANCE_ZK_HOST, zooKeepers);
-  }
-
-  /**
-   * Same as {@link #with(ClientProperty, String)} for ClientProperty.INSTANCE_ZK_TIMEOUT
-   *
-   */
-  public ClientConfiguration withZkTimeout(int timeout) {
-    return with(ClientProperty.INSTANCE_ZK_TIMEOUT, String.valueOf(timeout));
-  }
-
-  /**
-   * Same as {@link #withSsl(boolean, boolean)} with useJsseConfig set to false
-   *
-   */
-  public ClientConfiguration withSsl(boolean sslEnabled) {
-    return withSsl(sslEnabled, false);
-  }
-
-  /**
-   * Same as {@link #with(ClientProperty, String)} for ClientProperty.INSTANCE_RPC_SSL_ENABLED and
-   * ClientProperty.RPC_USE_JSSE
-   *
-   */
-  public ClientConfiguration withSsl(boolean sslEnabled, boolean useJsseConfig) {
-    return with(ClientProperty.INSTANCE_RPC_SSL_ENABLED, String.valueOf(sslEnabled))
-        .with(ClientProperty.RPC_USE_JSSE, String.valueOf(useJsseConfig));
-  }
-
-  /**
-   * Same as {@link #withTruststore(String, String, String)} with password null and type null
-   *
-   */
-  public ClientConfiguration withTruststore(String path) {
-    return withTruststore(path, null, null);
-  }
-
-  /**
-   * Same as {@link #with(ClientProperty, String)} for ClientProperty.RPC_SSL_TRUSTORE_PATH,
-   * ClientProperty.RPC_SSL_TRUSTORE_PASSWORD, and ClientProperty.RPC_SSL_TRUSTORE_TYPE
-   *
-   */
-  public ClientConfiguration withTruststore(String path, String password, String type) {
-    checkArgument(path != null, "path is null");
-    setProperty(ClientProperty.RPC_SSL_TRUSTSTORE_PATH, path);
-    if (password != null) {
-      setProperty(ClientProperty.RPC_SSL_TRUSTSTORE_PASSWORD, password);
-    }
-    if (type != null) {
-      setProperty(ClientProperty.RPC_SSL_TRUSTSTORE_TYPE, type);
-    }
-    return this;
-  }
-
-  /**
-   * Same as {@link #withKeystore(String, String, String)} with password null and type null
-   *
-   */
-  public ClientConfiguration withKeystore(String path) {
-    return withKeystore(path, null, null);
-  }
-
-  /**
-   * Same as {@link #with(ClientProperty, String)} for ClientProperty.INSTANCE_RPC_SSL_CLIENT_AUTH,
-   * ClientProperty.RPC_SSL_KEYSTORE_PATH, ClientProperty.RPC_SSL_KEYSTORE_PASSWORD, and
-   * ClientProperty.RPC_SSL_KEYSTORE_TYPE
-   *
-   */
-  public ClientConfiguration withKeystore(String path, String password, String type) {
-    checkArgument(path != null, "path is null");
-    setProperty(ClientProperty.INSTANCE_RPC_SSL_CLIENT_AUTH, "true");
-    setProperty(ClientProperty.RPC_SSL_KEYSTORE_PATH, path);
-    if (password != null) {
-      setProperty(ClientProperty.RPC_SSL_KEYSTORE_PASSWORD, password);
-    }
-    if (type != null) {
-      setProperty(ClientProperty.RPC_SSL_KEYSTORE_TYPE, type);
-    }
-    return this;
-  }
-
-  /**
-   * Same as {@link #with(ClientProperty, String)} for ClientProperty.INSTANCE_RPC_SASL_ENABLED.
-   *
-   * @since 1.7.0
-   */
-  public ClientConfiguration withSasl(boolean saslEnabled) {
-    return with(ClientProperty.INSTANCE_RPC_SASL_ENABLED, String.valueOf(saslEnabled));
-  }
-
-  /**
-   * Show whether SASL has been set on this configuration.
-   *
-   * @since 1.9.0
-   */
-  public boolean hasSasl() {
-    return compositeConfig.getBoolean(ClientProperty.INSTANCE_RPC_SASL_ENABLED.getKey(),
-        Boolean.parseBoolean(ClientProperty.INSTANCE_RPC_SASL_ENABLED.getDefaultValue()));
-  }
-
-  /**
-   * Same as {@link #with(ClientProperty, String)} for ClientProperty.INSTANCE_RPC_SASL_ENABLED and
-   * ClientProperty.GENERAL_KERBEROS_PRINCIPAL.
-   *
-   * @param saslEnabled Should SASL(kerberos) be enabled
-   * @param kerberosServerPrimary The 'primary' component of the Kerberos principal Accumulo servers
-   *        use to login (e.g. 'accumulo' in 'accumulo/_HOST@REALM')
-   * @since 1.7.0
-   */
-  public ClientConfiguration withSasl(boolean saslEnabled, String kerberosServerPrimary) {
-    return withSasl(saslEnabled).with(ClientProperty.KERBEROS_SERVER_PRIMARY,
-        kerberosServerPrimary);
-  }
-
-  public boolean containsKey(String key) {
-    return compositeConfig.containsKey(key);
-  }
-
-  public Iterator<String> getKeys() {
-    return compositeConfig.getKeys();
-  }
-
-  public String getString(String key) {
-    return compositeConfig.getString(key);
-  }
-
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/ClientSideIteratorScanner.java b/core/src/main/java/org/apache/accumulo/core/client/ClientSideIteratorScanner.java
index 362c4e8..7e300e9 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/ClientSideIteratorScanner.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/ClientSideIteratorScanner.java
@@ -22,6 +22,7 @@
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
 
 import java.io.IOException;
+import java.io.UncheckedIOException;
 import java.util.Collection;
 import java.util.Iterator;
 import java.util.Map;
@@ -52,7 +53,6 @@
 import org.apache.accumulo.core.iteratorsImpl.IteratorBuilder;
 import org.apache.accumulo.core.iteratorsImpl.IteratorConfigUtil;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.spi.common.ServiceEnvironment;
 import org.apache.hadoop.io.Text;
 
 /**
@@ -134,12 +134,6 @@
       return samplerConfig;
     }
 
-    @Deprecated(since = "2.1.0")
-    @Override
-    public ServiceEnvironment getServiceEnv() {
-      return new ClientServiceEnvironmentImpl(context.get());
-    }
-
     @Override
     public PluginEnvironment getPluginEnv() {
       return new ClientServiceEnvironmentImpl(context.get());
@@ -303,7 +297,7 @@
 
       skvi = IteratorConfigUtil.loadIterators(smi, ib);
     } catch (IOException e) {
-      throw new RuntimeException(e);
+      throw new UncheckedIOException(e);
     }
 
     final Set<ByteSequence> colfs = new TreeSet<>();
@@ -314,7 +308,7 @@
     try {
       skvi.seek(range, colfs, true);
     } catch (IOException e) {
-      throw new RuntimeException(e);
+      throw new UncheckedIOException(e);
     }
 
     return new IteratorAdapter(skvi);
diff --git a/core/src/main/java/org/apache/accumulo/core/client/Connector.java b/core/src/main/java/org/apache/accumulo/core/client/Connector.java
deleted file mode 100644
index 990824d..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/Connector.java
+++ /dev/null
@@ -1,283 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.client;
-
-import org.apache.accumulo.core.client.admin.InstanceOperations;
-import org.apache.accumulo.core.client.admin.NamespaceOperations;
-import org.apache.accumulo.core.client.admin.ReplicationOperations;
-import org.apache.accumulo.core.client.admin.SecurityOperations;
-import org.apache.accumulo.core.client.admin.TableOperations;
-import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.clientImpl.ConnectorImpl;
-import org.apache.accumulo.core.security.Authorizations;
-
-/**
- * Connector connects to an Accumulo instance and allows the user to request readers and writers for
- * the instance as well as various objects that permit administrative operations.
- *
- * The Connector enforces security on the client side by forcing all API calls to be accompanied by
- * user credentials.
- *
- * @deprecated since 2.0.0. Use {@link AccumuloClient} for writing new code. Connector is available
- *             for existing code. Use {@link #from(AccumuloClient)} as a bridge between the two.
- */
-@Deprecated(since = "2.0.0")
-public abstract class Connector {
-
-  /**
-   * Factory method to create a BatchScanner connected to Accumulo.
-   *
-   * @param tableName the name of the table to query
-   * @param authorizations A set of authorization labels that will be checked against the column
-   *        visibility of each key in order to filter data. The authorizations passed in must be a
-   *        subset of the accumulo user's set of authorizations. If the accumulo user has
-   *        authorizations (A1, A2) and authorizations (A2, A3) are passed, then an exception will
-   *        be thrown.
-   * @param numQueryThreads the number of concurrent threads to spawn for querying
-   *
-   * @return BatchScanner object for configuring and querying
-   * @throws TableNotFoundException when the specified table doesn't exist
-   */
-  public abstract BatchScanner createBatchScanner(String tableName, Authorizations authorizations,
-      int numQueryThreads) throws TableNotFoundException;
-
-  /**
-   * Factory method to create a BatchDeleter connected to Accumulo.
-   *
-   * @param tableName the name of the table to query and delete from
-   * @param authorizations A set of authorization labels that will be checked against the column
-   *        visibility of each key in order to filter data. The authorizations passed in must be a
-   *        subset of the accumulo user's set of authorizations. If the accumulo user has
-   *        authorizations (A1, A2) and authorizations (A2, A3) are passed, then an exception will
-   *        be thrown.
-   * @param numQueryThreads the number of concurrent threads to spawn for querying
-   * @param maxMemory size in bytes of the maximum memory to batch before writing
-   * @param maxLatency size in milliseconds; set to 0 or Long.MAX_VALUE to allow the maximum time to
-   *        hold a batch before writing
-   * @param maxWriteThreads the maximum number of threads to use for writing data to the tablet
-   *        servers
-   *
-   * @return BatchDeleter object for configuring and deleting
-   * @throws TableNotFoundException when the specified table doesn't exist
-   * @deprecated since 1.5.0; Use
-   *             {@link #createBatchDeleter(String, Authorizations, int, BatchWriterConfig)}
-   *             instead.
-   */
-  @Deprecated(since = "1.5.0")
-  public abstract BatchDeleter createBatchDeleter(String tableName, Authorizations authorizations,
-      int numQueryThreads, long maxMemory, long maxLatency, int maxWriteThreads)
-      throws TableNotFoundException;
-
-  /**
-   * Factory method to create BatchDeleter
-   *
-   * @param tableName the name of the table to query and delete from
-   * @param authorizations A set of authorization labels that will be checked against the column
-   *        visibility of each key in order to filter data. The authorizations passed in must be a
-   *        subset of the accumulo user's set of authorizations. If the accumulo user has
-   *        authorizations (A1, A2) and authorizations (A2, A3) are passed, then an exception will
-   *        be thrown.
-   * @param numQueryThreads the number of concurrent threads to spawn for querying
-   * @param config configuration used to create batch writer. This config takes precedence. Any
-   *        unset values will be merged with config set when the Connector was created. If no config
-   *        was set during Connector creation, BatchWriterConfig defaults will be used.
-   * @return BatchDeleter object for configuring and deleting
-   * @since 1.5.0
-   */
-  public abstract BatchDeleter createBatchDeleter(String tableName, Authorizations authorizations,
-      int numQueryThreads, BatchWriterConfig config) throws TableNotFoundException;
-
-  /**
-   * Factory method to create a BatchWriter connected to Accumulo.
-   *
-   * @param tableName the name of the table to insert data into
-   * @param maxMemory size in bytes of the maximum memory to batch before writing
-   * @param maxLatency time in milliseconds; set to 0 or Long.MAX_VALUE to allow the maximum time to
-   *        hold a batch before writing
-   * @param maxWriteThreads the maximum number of threads to use for writing data to the tablet
-   *        servers
-   *
-   * @return BatchWriter object for configuring and writing data to
-   * @throws TableNotFoundException when the specified table doesn't exist
-   * @deprecated since 1.5.0; Use {@link #createBatchWriter(String, BatchWriterConfig)} instead.
-   */
-  @Deprecated(since = "1.5.0")
-  public abstract BatchWriter createBatchWriter(String tableName, long maxMemory, long maxLatency,
-      int maxWriteThreads) throws TableNotFoundException;
-
-  /**
-   * Factory method to create a BatchWriter connected to Accumulo.
-   *
-   * @param tableName the name of the table to insert data into
-   * @param config configuration used to create batch writer. This config will take precedence. Any
-   *        unset values will merged with config set when the Connector was created. If no config
-   *        was set during Connector creation, BatchWriterConfig defaults will be used.
-   * @return BatchWriter object for configuring and writing data to
-   * @since 1.5.0
-   */
-  public abstract BatchWriter createBatchWriter(String tableName, BatchWriterConfig config)
-      throws TableNotFoundException;
-
-  /**
-   * Factory method to create a Multi-Table BatchWriter connected to Accumulo. Multi-table batch
-   * writers can queue data for multiple tables, which is good for ingesting data into multiple
-   * tables from the same source
-   *
-   * @param maxMemory size in bytes of the maximum memory to batch before writing
-   * @param maxLatency size in milliseconds; set to 0 or Long.MAX_VALUE to allow the maximum time to
-   *        hold a batch before writing
-   * @param maxWriteThreads the maximum number of threads to use for writing data to the tablet
-   *        servers
-   *
-   * @return MultiTableBatchWriter object for configuring and writing data to
-   * @deprecated since 1.5.0; Use {@link #createMultiTableBatchWriter(BatchWriterConfig)} instead.
-   */
-  @Deprecated(since = "1.5.0")
-  public abstract MultiTableBatchWriter createMultiTableBatchWriter(long maxMemory, long maxLatency,
-      int maxWriteThreads);
-
-  /**
-   * Factory method to create a Multi-Table BatchWriter connected to Accumulo. Multi-table batch
-   * writers can queue data for multiple tables. Also data for multiple tables can be sent to a
-   * server in a single batch. Its an efficient way to ingest data into multiple tables from a
-   * single process.
-   *
-   * @param config configuration used to create multi-table batch writer. This config will take
-   *        precedence. Any unset values will merged with config set when the Connector was created.
-   *        If no config was set during Connector creation, BatchWriterConfig defaults will be used.
-   * @return MultiTableBatchWriter object for configuring and writing data to
-   * @since 1.5.0
-   */
-  public abstract MultiTableBatchWriter createMultiTableBatchWriter(BatchWriterConfig config);
-
-  /**
-   * Factory method to create a Scanner connected to Accumulo.
-   *
-   * @param tableName the name of the table to query data from
-   * @param authorizations A set of authorization labels that will be checked against the column
-   *        visibility of each key in order to filter data. The authorizations passed in must be a
-   *        subset of the accumulo user's set of authorizations. If the accumulo user has
-   *        authorizations (A1, A2) and authorizations (A2, A3) are passed, then an exception will
-   *        be thrown.
-   *
-   * @return Scanner object for configuring and querying data with
-   * @throws TableNotFoundException when the specified table doesn't exist
-   */
-  public abstract Scanner createScanner(String tableName, Authorizations authorizations)
-      throws TableNotFoundException;
-
-  /**
-   * Factory method to create a ConditionalWriter connected to Accumulo.
-   *
-   * @param tableName the name of the table to query data from
-   * @param config configuration used to create conditional writer
-   *
-   * @return ConditionalWriter object for writing ConditionalMutations
-   * @throws TableNotFoundException when the specified table doesn't exist
-   * @since 1.6.0
-   */
-  public abstract ConditionalWriter createConditionalWriter(String tableName,
-      ConditionalWriterConfig config) throws TableNotFoundException;
-
-  /**
-   * Accessor method for internal instance object.
-   *
-   * @return the internal instance object
-   */
-  public abstract Instance getInstance();
-
-  /**
-   * Get the current user for this connector
-   *
-   * @return the user name
-   */
-  public abstract String whoami();
-
-  /**
-   * Retrieves a TableOperations object to perform table functions, such as create and delete.
-   *
-   * @return an object to manipulate tables
-   */
-  public abstract TableOperations tableOperations();
-
-  /**
-   * Retrieves a NamespaceOperations object to perform namespace functions, such as create and
-   * delete.
-   *
-   * @return an object to manipulate namespaces
-   */
-  public abstract NamespaceOperations namespaceOperations();
-
-  /**
-   * Retrieves a SecurityOperations object to perform user security operations, such as creating
-   * users.
-   *
-   * @return an object to modify users and permissions
-   */
-  public abstract SecurityOperations securityOperations();
-
-  /**
-   * Retrieves an InstanceOperations object to modify instance configuration.
-   *
-   * @return an object to modify instance configuration
-   */
-  public abstract InstanceOperations instanceOperations();
-
-  /**
-   * Retrieves a ReplicationOperations object to manage replication configuration.
-   *
-   * @return an object to modify replication configuration
-   * @since 1.7.0
-   */
-  public abstract ReplicationOperations replicationOperations();
-
-  /**
-   * Creates a Connector from an AccumuloClient. This Connector will no longer work after the
-   * AccumuloClient is closed. Also anything derived from the Connector (like a Scanner for example)
-   * is unlikely to work after the AccumuloClient is closed.
-   *
-   * @since 2.0
-   */
-  public static Connector from(AccumuloClient client)
-      throws AccumuloSecurityException, AccumuloException {
-    return new ConnectorImpl((ClientContext) client);
-  }
-
-  /**
-   * Creates a new Accumulo Client from a Connector. The returned client should be closed and
-   * closing it will not affect the Connector from which it was derived. This method is useful for
-   * cases where code written using Connector must call code written using AccumuloClient. Below is
-   * an example.
-   *
-   * <pre>
-   * <code>
-   *   Connector conn = getMyConnector();
-   *   try(AccumuloClient client = Connector.newClient(conn) {
-   *      doSomething(client);
-   *   }
-   * </code>
-   * </pre>
-   *
-   * @since 2.1.0
-   */
-  public static AccumuloClient newClient(Connector conn) {
-    return Accumulo.newClient().from(((ConnectorImpl) conn).getAccumuloClient().getProperties())
-        .build();
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/Instance.java b/core/src/main/java/org/apache/accumulo/core/client/Instance.java
deleted file mode 100644
index 947f8b1..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/Instance.java
+++ /dev/null
@@ -1,139 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.client;
-
-import java.nio.ByteBuffer;
-import java.util.List;
-
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-
-/**
- * This class represents the information a client needs to know to connect to an instance of
- * accumulo.
- *
- * @deprecated since 2.0.0, use {@link Accumulo#newClient()} and {@link java.util.Properties}
- *             instead
- */
-@Deprecated(since = "2.0.0")
-public interface Instance {
-  /**
-   * Returns the location of the tablet server that is serving the root tablet.
-   *
-   * @return location in "hostname:port" form
-   */
-  String getRootTabletLocation();
-
-  /**
-   * Returns the location(s) of the accumulo manager and any redundant servers.
-   *
-   * @return a list of locations in "hostname:port" form
-   */
-  List<String> getMasterLocations();
-
-  /**
-   * Returns a unique string that identifies this instance of accumulo.
-   *
-   * @return a UUID
-   */
-  String getInstanceID();
-
-  /**
-   * Returns the instance name given at system initialization time.
-   *
-   * @return current instance name
-   */
-  String getInstanceName();
-
-  /**
-   * Returns a comma-separated list of zookeeper servers the instance is using.
-   *
-   * @return the zookeeper servers this instance is using in "hostname:port" form
-   */
-  String getZooKeepers();
-
-  /**
-   * Returns the zookeeper connection timeout.
-   *
-   * @return the configured timeout to connect to zookeeper
-   */
-  int getZooKeepersSessionTimeOut();
-
-  /**
-   * Returns a connection to accumulo.
-   *
-   * @param user a valid accumulo user
-   * @param pass A UTF-8 encoded password. The password may be cleared after making this call.
-   * @return the accumulo Connector
-   * @throws AccumuloException when a generic exception occurs
-   * @throws AccumuloSecurityException when a user's credentials are invalid
-   * @deprecated since 1.5, use {@link #getConnector(String, AuthenticationToken)} with
-   *             {@link PasswordToken}
-   */
-  @Deprecated(since = "1.5.0")
-  default Connector getConnector(String user, byte[] pass)
-      throws AccumuloException, AccumuloSecurityException {
-    return getConnector(user, new PasswordToken(pass));
-  }
-
-  /**
-   * Returns a connection to accumulo.
-   *
-   * @param user a valid accumulo user
-   * @param pass A UTF-8 encoded password. The password may be cleared after making this call.
-   * @return the accumulo Connector
-   * @throws AccumuloException when a generic exception occurs
-   * @throws AccumuloSecurityException when a user's credentials are invalid
-   * @deprecated since 1.5, use {@link #getConnector(String, AuthenticationToken)} with
-   *             {@link PasswordToken}
-   */
-  @Deprecated(since = "1.5.0")
-  default Connector getConnector(String user, ByteBuffer pass)
-      throws AccumuloException, AccumuloSecurityException {
-    return getConnector(user, new PasswordToken(pass));
-  }
-
-  /**
-   * Returns a connection to this instance of accumulo.
-   *
-   * @param user a valid accumulo user
-   * @param pass If a mutable CharSequence is passed in, it may be cleared after this call.
-   * @return the accumulo Connector
-   * @throws AccumuloException when a generic exception occurs
-   * @throws AccumuloSecurityException when a user's credentials are invalid
-   * @deprecated since 1.5, use {@link #getConnector(String, AuthenticationToken)} with
-   *             {@link PasswordToken}
-   */
-  @Deprecated(since = "1.5.0")
-  default Connector getConnector(String user, CharSequence pass)
-      throws AccumuloException, AccumuloSecurityException {
-    return getConnector(user, new PasswordToken(pass));
-  }
-
-  /**
-   * Returns a connection to this instance of accumulo.
-   *
-   * @param principal a valid accumulo user
-   * @param token Use the token type configured for the Accumulo instance you are connecting to. An
-   *        Accumulo instance with default configurations will use {@link PasswordToken}
-   * @since 1.5.0
-   */
-  Connector getConnector(String principal, AuthenticationToken token)
-      throws AccumuloException, AccumuloSecurityException;
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/IsolatedScanner.java b/core/src/main/java/org/apache/accumulo/core/client/IsolatedScanner.java
index 518737b..c3ead89 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/IsolatedScanner.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/IsolatedScanner.java
@@ -18,8 +18,8 @@
  */
 package org.apache.accumulo.core.client;
 
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
 
 import java.util.ArrayList;
 import java.util.Iterator;
diff --git a/core/src/main/java/org/apache/accumulo/core/client/MutationsRejectedException.java b/core/src/main/java/org/apache/accumulo/core/client/MutationsRejectedException.java
index 30e6fbd..1603268 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/MutationsRejectedException.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/MutationsRejectedException.java
@@ -45,34 +45,6 @@
   private final int unknownErrors;
 
   /**
-   *
-   * @param cvsList list of constraint violations
-   * @param hashMap authorization failures
-   * @param serverSideErrors server side errors
-   * @param unknownErrors number of unknown errors
-   *
-   * @since 1.7.0
-   * @deprecated since 2.0.0, replaced by
-   *             {@link #MutationsRejectedException(AccumuloClient, List, Map, Collection, int, Throwable)}
-   */
-  @Deprecated(since = "2.0.0")
-  public MutationsRejectedException(Instance instance, List<ConstraintViolationSummary> cvsList,
-      Map<TabletId,Set<SecurityErrorCode>> hashMap, Collection<String> serverSideErrors,
-      int unknownErrors, Throwable cause) {
-    super(
-        "constraint violation codes : "
-            + cvsList.stream().map(ConstraintViolationSummary::getViolationCode)
-                .collect(Collectors.toSet())
-            + "  security codes: " + hashMap.toString() + "  # server errors "
-            + serverSideErrors.size() + " # exceptions " + unknownErrors,
-        cause);
-    this.cvsl.addAll(cvsList);
-    this.af.putAll(hashMap);
-    this.es.addAll(serverSideErrors);
-    this.unknownErrors = unknownErrors;
-  }
-
-  /**
    * Creates Mutations rejected exception
    *
    * @param client AccumuloClient
diff --git a/core/src/main/java/org/apache/accumulo/core/client/PluginEnvironment.java b/core/src/main/java/org/apache/accumulo/core/client/PluginEnvironment.java
index 6653c80..4372a15 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/PluginEnvironment.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/PluginEnvironment.java
@@ -163,7 +163,7 @@
    * @param className Fully qualified name of the class.
    * @param base The expected super type of the class.
    */
-  <T> T instantiate(String className, Class<T> base) throws Exception;
+  <T> T instantiate(String className, Class<T> base) throws ReflectiveOperationException;
 
   /**
    * Instantiate a class using Accumulo's per table classloader. The class must have a no argument
@@ -172,5 +172,6 @@
    * @param className Fully qualified name of the class.
    * @param base The expected super type of the class.
    */
-  <T> T instantiate(TableId tableId, String className, Class<T> base) throws Exception;
+  <T> T instantiate(TableId tableId, String className, Class<T> base)
+      throws ReflectiveOperationException;
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/client/TableOfflineException.java b/core/src/main/java/org/apache/accumulo/core/client/TableOfflineException.java
index 2dbeb14..bd0e266 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/TableOfflineException.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/TableOfflineException.java
@@ -25,14 +25,6 @@
   private static final long serialVersionUID = 1L;
 
   /**
-   * @deprecated since 2.0.0, replaced by {@link #TableOfflineException(String)}
-   */
-  @Deprecated(since = "2.0.0")
-  public TableOfflineException(Instance instance, String tableId) {
-    super("Table with ID (" + tableId + ") is offline");
-  }
-
-  /**
    * @since 2.0.0
    */
   public TableOfflineException(String msg) {
diff --git a/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java b/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
deleted file mode 100644
index aa96404..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
+++ /dev/null
@@ -1,203 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.client;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static java.util.concurrent.TimeUnit.SECONDS;
-
-import java.util.List;
-import java.util.Properties;
-
-import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.clientImpl.ClientConfConverter;
-import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.clientImpl.ClientInfo;
-import org.apache.accumulo.core.clientImpl.ClientInfoImpl;
-import org.apache.accumulo.core.clientImpl.InstanceOperationsImpl;
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.ClientProperty;
-import org.apache.accumulo.core.conf.ConfigurationTypeHelper;
-import org.apache.accumulo.core.data.InstanceId;
-import org.apache.accumulo.core.fate.zookeeper.ZooCache;
-import org.apache.accumulo.core.fate.zookeeper.ZooCacheFactory;
-import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
-import org.apache.accumulo.core.metadata.schema.TabletMetadata.LocationType;
-import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
-import org.apache.accumulo.core.singletons.SingletonManager;
-import org.apache.accumulo.core.singletons.SingletonManager.Mode;
-import org.apache.accumulo.core.singletons.SingletonReservation;
-import org.apache.accumulo.core.util.OpTimer;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * An implementation of instance that looks in zookeeper to find information needed to connect to an
- * instance of accumulo.
- *
- * <p>
- * The advantage of using zookeeper to obtain information about accumulo is that zookeeper is highly
- * available, very responsive, and supports caching.
- *
- * <p>
- * Because it is possible for multiple instances of accumulo to share a single set of zookeeper
- * servers, all constructors require an accumulo instance name.
- *
- * If you do not know the instance names then run accumulo
- * org.apache.accumulo.server.util.ListInstances on an accumulo server.
- *
- * @deprecated since 2.0.0, Use {@link Accumulo#newClient()} instead
- */
-@Deprecated(since = "2.0.0")
-public class ZooKeeperInstance implements Instance {
-
-  private static final Logger log = LoggerFactory.getLogger(ZooKeeperInstance.class);
-
-  private String instanceId = null;
-  private String instanceName = null;
-
-  private final ZooCache zooCache;
-
-  private final String zooKeepers;
-
-  private final int zooKeepersSessionTimeOut;
-
-  private ClientConfiguration clientConf;
-
-  /**
-   *
-   * @param instanceName The name of specific accumulo instance. This is set at initialization time.
-   * @param zooKeepers A comma separated list of zoo keeper server locations. Each location can
-   *        contain an optional port, of the format host:port.
-   */
-  public ZooKeeperInstance(String instanceName, String zooKeepers) {
-    this(ClientConfiguration.loadDefault().withInstance(instanceName).withZkHosts(zooKeepers));
-  }
-
-  ZooKeeperInstance(ClientConfiguration config, ZooCacheFactory zcf) {
-    checkArgument(config != null, "config is null");
-    // Enable singletons before before getting a zoocache
-    SingletonManager.setMode(Mode.CONNECTOR);
-    this.clientConf = config;
-    this.instanceId = clientConf.get(ClientConfiguration.ClientProperty.INSTANCE_ID);
-    this.instanceName = clientConf.get(ClientConfiguration.ClientProperty.INSTANCE_NAME);
-    if ((instanceId == null) == (instanceName == null)) {
-      throw new IllegalArgumentException(
-          "Expected exactly one of instanceName and instanceId to be set; "
-              + (instanceName == null ? "neither" : "both") + " were set");
-    }
-    this.zooKeepers = clientConf.get(ClientConfiguration.ClientProperty.INSTANCE_ZK_HOST);
-    this.zooKeepersSessionTimeOut = (int) ConfigurationTypeHelper
-        .getTimeInMillis(clientConf.get(ClientConfiguration.ClientProperty.INSTANCE_ZK_TIMEOUT));
-    zooCache = zcf.getZooCache(zooKeepers, zooKeepersSessionTimeOut);
-    if (instanceName != null) {
-      // Validates that the provided instanceName actually exists
-      getInstanceID();
-    }
-  }
-
-  /**
-   * @param config Client configuration for specifying connection options. See
-   *        {@link ClientConfiguration} which extends Configuration with convenience methods
-   *        specific to Accumulo.
-   * @since 1.9.0
-   */
-  public ZooKeeperInstance(ClientConfiguration config) {
-    this(config, new ZooCacheFactory());
-  }
-
-  @Override
-  public String getInstanceID() {
-    if (instanceId == null) {
-      instanceId = ClientContext.getInstanceID(zooCache, instanceName).canonical();
-    }
-    ClientContext.verifyInstanceId(zooCache, instanceId, instanceName);
-    return instanceId;
-  }
-
-  @Override
-  public List<String> getMasterLocations() {
-    return ClientContext.getManagerLocations(zooCache, getInstanceID());
-  }
-
-  @Override
-  public String getRootTabletLocation() {
-    OpTimer timer = null;
-
-    if (log.isTraceEnabled()) {
-      log.trace("tid={} Looking up root tablet location in zookeeper.",
-          Thread.currentThread().getId());
-      timer = new OpTimer().start();
-    }
-
-    Location loc = TabletsMetadata
-        .getRootMetadata(Constants.ZROOT + "/" + getInstanceID(), zooCache).getLocation();
-
-    if (timer != null) {
-      timer.stop();
-      log.trace("tid={} Found root tablet at {} in {}", Thread.currentThread().getId(), loc,
-          String.format("%.3f secs", timer.scale(SECONDS)));
-    }
-
-    if (loc == null || loc.getType() != LocationType.CURRENT) {
-      return null;
-    }
-
-    return loc.getHostPort();
-  }
-
-  @Override
-  public String getInstanceName() {
-    if (instanceName == null) {
-      instanceName =
-          InstanceOperationsImpl.lookupInstanceName(zooCache, InstanceId.of(getInstanceID()));
-    }
-
-    return instanceName;
-  }
-
-  @Override
-  public String getZooKeepers() {
-    return zooKeepers;
-  }
-
-  @Override
-  public int getZooKeepersSessionTimeOut() {
-    return zooKeepersSessionTimeOut;
-  }
-
-  @Override
-  public Connector getConnector(String principal, AuthenticationToken token)
-      throws AccumuloException, AccumuloSecurityException {
-    Properties properties = ClientConfConverter.toProperties(clientConf);
-    properties.setProperty(ClientProperty.AUTH_PRINCIPAL.getKey(), principal);
-    properties.setProperty(ClientProperty.INSTANCE_NAME.getKey(), getInstanceName());
-    ClientInfo info = new ClientInfoImpl(properties, token);
-    AccumuloConfiguration serverConf = ClientConfConverter.toAccumuloConf(properties);
-    return new org.apache.accumulo.core.clientImpl.ConnectorImpl(
-        new ClientContext(SingletonReservation.noop(), info, serverConf, null));
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder sb = new StringBuilder(64);
-    sb.append("ZooKeeperInstance: ").append(getInstanceName()).append(" ").append(getZooKeepers());
-    return sb.toString();
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveCompaction.java b/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveCompaction.java
index 45a31e9..dd02f6b 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveCompaction.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveCompaction.java
@@ -29,17 +29,12 @@
  */
 public abstract class ActiveCompaction {
 
-  public static enum CompactionType {
+  public enum CompactionType {
     /**
      * compaction to flush a tablets memory
      */
     MINOR,
     /**
-     * Accumulo no longer does merging minor compactions.
-     */
-    @Deprecated(since = "2.1.0", forRemoval = true)
-    MERGE,
-    /**
      * compaction that merges a subset of a tablets files into one file
      */
     MAJOR,
@@ -49,7 +44,7 @@
     FULL
   }
 
-  public static enum CompactionReason {
+  public enum CompactionReason {
     /**
      * compaction initiated by user
      */
@@ -59,8 +54,10 @@
      */
     SYSTEM,
     /**
-     * Compaction initiated by merge operation
+     * @deprecated Chop compactions no longer occur and it's not expected that listing compaction
+     *             would ever return this.
      */
+    @Deprecated(since = "3.1")
     CHOP,
     /**
      * idle compaction
@@ -125,6 +122,12 @@
   public abstract long getEntriesWritten();
 
   /**
+   * @return the number of times the server paused a compaction
+   * @since 3.0.0
+   */
+  public abstract long getPausedCount();
+
+  /**
    * @return the per compaction iterators configured
    */
   public abstract List<IteratorSetting> getIterators();
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/CompactionConfig.java b/core/src/main/java/org/apache/accumulo/core/client/admin/CompactionConfig.java
index 64392a1..1600ea0 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/CompactionConfig.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/CompactionConfig.java
@@ -19,7 +19,6 @@
 package org.apache.accumulo.core.client.admin;
 
 import static java.util.Objects.requireNonNull;
-import static org.apache.accumulo.core.clientImpl.CompactionStrategyConfigUtil.DEFAULT_STRATEGY;
 
 import java.util.Collections;
 import java.util.List;
@@ -29,7 +28,6 @@
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.admin.compaction.CompactionConfigurer;
 import org.apache.accumulo.core.client.admin.compaction.CompactionSelector;
-import org.apache.accumulo.core.clientImpl.CompactionStrategyConfigUtil;
 import org.apache.accumulo.core.clientImpl.UserCompactionUtils;
 import org.apache.hadoop.io.Text;
 
@@ -40,7 +38,6 @@
  *
  * @since 1.7.0
  */
-@SuppressWarnings("removal")
 public class CompactionConfig {
 
   private Text start = null;
@@ -48,7 +45,6 @@
   private boolean flush = true;
   private boolean wait = true;
   private List<IteratorSetting> iterators = Collections.emptyList();
-  private CompactionStrategyConfig compactionStrategy = DEFAULT_STRATEGY;
   private Map<String,String> hints = Map.of();
   private PluginConfig selectorConfig = UserCompactionUtils.DEFAULT_SELECTOR;
   private PluginConfig configurerConfig = UserCompactionUtils.DEFAULT_CONFIGURER;
@@ -144,35 +140,6 @@
   }
 
   /**
-   * @param csConfig configures the strategy that will be used by each tablet to select files. If no
-   *        strategy is set, then all files will be compacted.
-   * @return this
-   * @deprecated since 2.1.0 use {@link #setSelector(PluginConfig)} and
-   *             {@link #setConfigurer(PluginConfig)} instead. See {@link CompactionStrategyConfig}
-   *             for details about why this was deprecated.
-   */
-  @Deprecated(since = "2.1.0", forRemoval = true)
-  public CompactionConfig setCompactionStrategy(CompactionStrategyConfig csConfig) {
-    requireNonNull(csConfig);
-    Preconditions.checkArgument(!csConfig.getClassName().isBlank());
-    Preconditions.checkState(
-        selectorConfig.getClassName().isEmpty() && configurerConfig.getClassName().isEmpty());
-    this.compactionStrategy = csConfig;
-    return this;
-  }
-
-  /**
-   * @return The previously set compaction strategy. Defaults to a configuration of
-   *         org.apache.accumulo.tserver.compaction.EverythingCompactionStrategy which always
-   *         compacts all files.
-   * @deprecated since 2.1.0
-   */
-  @Deprecated(since = "2.1.0", forRemoval = true)
-  public CompactionStrategyConfig getCompactionStrategy() {
-    return compactionStrategy;
-  }
-
-  /**
    * Configure a {@link CompactionSelector} plugin to run for this compaction. Specify the class
    * name and options here.
    *
@@ -180,7 +147,6 @@
    * @since 2.1.0
    */
   public CompactionConfig setSelector(PluginConfig selectorConfig) {
-    Preconditions.checkState(compactionStrategy.getClassName().isEmpty());
     Preconditions.checkArgument(!selectorConfig.getClassName().isBlank());
     this.selectorConfig = requireNonNull(selectorConfig);
     return this;
@@ -197,9 +163,6 @@
    * @since 2.1.0
    */
   public CompactionConfig setExecutionHints(Map<String,String> hints) {
-    if (!hints.isEmpty()) {
-      Preconditions.checkState(compactionStrategy.getClassName().isEmpty());
-    }
     this.hints = Map.copyOf(hints);
     return this;
   }
@@ -218,7 +181,6 @@
    * @since 2.1.0
    */
   public CompactionConfig setConfigurer(PluginConfig configurerConfig) {
-    Preconditions.checkState(compactionStrategy.getClassName().isEmpty());
     this.configurerConfig = configurerConfig;
     return this;
   }
@@ -251,8 +213,6 @@
     prefix = append(sb, prefix, () -> !flush, "flush", flush);
     prefix = append(sb, prefix, () -> !wait, "wait", wait);
     prefix = append(sb, prefix, () -> !iterators.isEmpty(), "iterators", iterators);
-    prefix = append(sb, prefix, () -> !CompactionStrategyConfigUtil.isDefault(compactionStrategy),
-        "strategy", compactionStrategy);
     prefix = append(sb, prefix, () -> !UserCompactionUtils.isDefault(selectorConfig), "selector",
         selectorConfig);
     prefix = append(sb, prefix, () -> !UserCompactionUtils.isDefault(configurerConfig),
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/CompactionStrategyConfig.java b/core/src/main/java/org/apache/accumulo/core/client/admin/CompactionStrategyConfig.java
deleted file mode 100644
index 2f880a6..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/CompactionStrategyConfig.java
+++ /dev/null
@@ -1,96 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.client.admin;
-
-import static java.util.Objects.requireNonNull;
-
-import java.util.Map;
-
-/**
- * Configuration object which describes how a Compaction is run. Configuration objects are dependent
- * upon the CompactionStrategy running insider the server. This class is used in conjunction with
- * {@link CompactionConfig}.
- *
- * @since 1.7.0
- * @deprecated since 2.1.0 CompactionStrategies were deprecated for multiple reasons. First, they do
- *             not support the new compaction execution model. Second, they bind selection and
- *             output file configuration into a single entity when users need to configure these
- *             independently. Third, they use internal Accumulo types and ensuring their stability
- *             requires manual effort that may never happen. Fourth, writing a correct compaction
- *             strategy was exceedingly difficult as it required knowledge of internal tablet server
- *             synchronization in order to avoid causing scans to hang. Fifth although measure were
- *             taken to execute compaction strategies in the same manner as before, their execution
- *             in the new model has subtle differences that may result in suboptimal compactions.
- *             Please migrate to using {@link CompactionConfig#setSelector(PluginConfig)} and
- *             {@link CompactionConfig#setConfigurer(PluginConfig)} as soon as possible.
- */
-@Deprecated(since = "2.1.0", forRemoval = true)
-public class CompactionStrategyConfig {
-  private String className;
-  private Map<String,String> options = Map.of();
-
-  /**
-   * @param className The name of a class that implements
-   *        org.apache.accumulo.tserver.compaction.CompactionStrategy. This class must exist on
-   *        tservers.
-   */
-  public CompactionStrategyConfig(String className) {
-    this.className = requireNonNull(className);
-  }
-
-  /**
-   * @return the class name passed to the constructor.
-   */
-  public String getClassName() {
-    return className;
-  }
-
-  /**
-   * @param opts The options that will be passed to the init() method of the compaction strategy
-   *        when its instantiated on a tserver. This method will copy the map. The default is an
-   *        empty map.
-   * @return this
-   */
-  public CompactionStrategyConfig setOptions(Map<String,String> opts) {
-    this.options = Map.copyOf(opts);
-    return this;
-  }
-
-  /**
-   * @return The previously set options. Returns an unmodifiable map. The default is an empty map.
-   */
-  public Map<String,String> getOptions() {
-    return options;
-  }
-
-  @Override
-  public int hashCode() {
-    return className.hashCode() + options.hashCode();
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (o instanceof CompactionStrategyConfig) {
-      CompactionStrategyConfig ocsc = (CompactionStrategyConfig) o;
-      return className.equals(ocsc.className) && options.equals(ocsc.options);
-    }
-
-    return false;
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/FindMax.java b/core/src/main/java/org/apache/accumulo/core/client/admin/FindMax.java
index 2caba85..a953d7d 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/FindMax.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/FindMax.java
@@ -67,7 +67,7 @@
 
     if (ba.length == startOS.size()) {
       if (ba[0] != 0) {
-        throw new RuntimeException();
+        throw new IllegalStateException();
       }
 
       // big int added a zero so it would not be negative, drop it
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 2e1c0c9..4cb3f6d 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
@@ -87,22 +87,22 @@
    *         {@code
    *             AccumuloClient client = getClient();
    *             Map<String,String> acceptedProps = client.instanceOperations().modifyProperties(currProps -> {
-   *               var planner = currProps.get("tserver.compaction.major.service.default.planner");
+   *               var planner = currProps.get("compaction.service.default.planner");
    *               //This code will only change the compaction planner if its currently set to default settings.
    *               //The endsWith() function was used to make the example short, would be better to use equals().
    *               if(planner != null && planner.endsWith("DefaultCompactionPlanner") {
    *                 // tservers will eventually see these compaction planner changes and when they do they will see all of the changes at once
    *                 currProps.keySet().removeIf(
-   *                    prop -> prop.startsWith("tserver.compaction.major.service.default.planner.opts."));
-   *                 currProps.put("tserver.compaction.major.service.default.planner","MyPlannerClassName");
-   *                 currProps.put("tserver.compaction.major.service.default.planner.opts.myOpt1","val1");
-   *                 currProps.put("tserver.compaction.major.service.default.planner.opts.myOpt2","val2");
+   *                    prop -> prop.startsWith("compaction.service.default.planner.opts."));
+   *                 currProps.put("compaction.service.default.planner","MyPlannerClassName");
+   *                 currProps.put("compaction.service.default.planner.opts.myOpt1","val1");
+   *                 currProps.put("compaction.service.default.planner.opts.myOpt2","val2");
    *                }
    *             });
    *
    *             // Since three properties were set may want to check for the values of all
    *             // three, just checking one in this example to keep it short.
-   *             if("MyPlannerClassName".equals(acceptedProps.get("tserver.compaction.major.service.default.planner"))){
+   *             if("MyPlannerClassName".equals(acceptedProps.get("compaction.service.default.planner"))){
    *                // the compaction planner change was accepted or already existed, so take action for that outcome
    *             } else {
    *                // the compaction planner change was not done, so take action for that outcome
@@ -140,7 +140,17 @@
   void removeProperty(final String property) throws AccumuloException, AccumuloSecurityException;
 
   /**
-   * Retrieve the system-wide configuration.
+   * Retrieve the system-wide, merged view of the system configuration. Accumulo has multiple layers
+   * of properties, in order of precedence (highest - lowest):
+   * <ul>
+   * <li>the properties set in Zookeeper</li>
+   * <li>the properties set in the site configuration file</li>
+   * <li>the default properties</li>
+   * </ul>
+   * The properties returned is the merged view of these properties. The properties that are stored
+   * in ZooKeeper can be modified with {@link #modifyProperties modifyProperties},
+   * {@link #setProperty setProperty} and {@link #removeProperty removeProperty}. Properties can be
+   * further refined by namesapce {@link NamespaceOperations} and by table {@link TableOperations}.
    *
    * @return A map of system properties set in zookeeper. If a property is not set in zookeeper,
    *         then it will return the value set in accumulo.properties on some server. If nothing is
@@ -157,6 +167,16 @@
   Map<String,String> getSiteConfiguration() throws AccumuloException, AccumuloSecurityException;
 
   /**
+   * Retrieve a map of the system properties set in Zookeeper. Note that this does not return a
+   * merged view of the properties from its parent configuration. See
+   * {@link #getSystemConfiguration} for a merged view.
+   *
+   * @return A map of the system properties set in Zookeeper only.
+   * @since 3.1
+   */
+  Map<String,String> getSystemProperties() throws AccumuloException, AccumuloSecurityException;
+
+  /**
    * Returns the location(s) of the accumulo manager and any redundant servers.
    *
    * @return a list of locations in <code>hostname:port</code> form.
@@ -237,17 +257,6 @@
   void waitForBalance() throws AccumuloException;
 
   /**
-   * Returns a unique string that identifies this instance of accumulo.
-   *
-   * @return a String
-   * @since 2.0.0
-   *
-   * @deprecated in 2.1.0 Use {@link #getInstanceId()}
-   */
-  @Deprecated(since = "2.1.0")
-  String getInstanceID();
-
-  /**
    * Returns a unique ID object that identifies this instance of accumulo.
    *
    * @return an InstanceId
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/ReplicationOperations.java b/core/src/main/java/org/apache/accumulo/core/client/admin/ReplicationOperations.java
deleted file mode 100644
index 2815174..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/ReplicationOperations.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.client.admin;
-
-import java.util.Set;
-
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.replication.PeerExistsException;
-import org.apache.accumulo.core.client.replication.PeerNotFoundException;
-
-/**
- * Supports replication configuration
- *
- * @since 1.7.0
- */
-@Deprecated
-public interface ReplicationOperations {
-
-  /**
-   * Defines a cluster with the given name and the given name system.
-   *
-   * @param name Unique name for the cluster
-   * @param replicaType Class name to use to replicate the data
-   */
-  void addPeer(String name, String replicaType)
-      throws AccumuloException, AccumuloSecurityException, PeerExistsException;
-
-  /**
-   * Removes a cluster with the given name.
-   *
-   * @param name Name of the cluster to remove
-   */
-  void removePeer(String name)
-      throws AccumuloException, AccumuloSecurityException, PeerNotFoundException;
-
-  /**
-   * Waits for a table to be fully replicated, given the state of files pending replication for the
-   * provided table at the point in time which this method is invoked.
-   *
-   * @param tableName The table to wait for
-   */
-  void drain(String tableName)
-      throws AccumuloException, AccumuloSecurityException, TableNotFoundException;
-
-  /**
-   * Given the provided set of files that are pending replication for a table, wait for those files
-   * to be fully replicated to all configured peers. This allows for the accurate calculation when a
-   * table, at a given point in time, has been fully replicated.
-   *
-   * @param tableName The table to wait for
-   */
-  void drain(String tableName, Set<String> files)
-      throws AccumuloException, AccumuloSecurityException, TableNotFoundException;
-
-  /**
-   * Gets all of the referenced files for a table from the metadata table. The result of this method
-   * is intended to be directly supplied to {@link #drain(String, Set)}. This helps determine when
-   * all data from a given point in time has been fully replicated.
-   * <p>
-   * This also allows callers to get the {@link Set} of files for a table at some time, and later
-   * provide that {@link Set} to {@link #drain(String,Set)} to wait for all of those files to be
-   * replicated.
-   */
-  Set<String> referencedFiles(String tableName)
-      throws AccumuloException, AccumuloSecurityException, TableNotFoundException;
-}
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 eabafaa..fef5599 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
@@ -88,47 +88,6 @@
       throws AccumuloException, AccumuloSecurityException, TableExistsException;
 
   /**
-   * @param tableName the name of the table
-   * @param limitVersion Enables/disables the versioning iterator, which will limit the number of
-   *        Key versions kept.
-   * @throws AccumuloException if a general error occurs
-   * @throws AccumuloSecurityException if the user does not have permission
-   * @throws TableExistsException if the table already exists
-   * @deprecated since 1.7.0; use {@link #create(String, NewTableConfiguration)} instead.
-   */
-  @Deprecated(since = "1.7.0")
-  default void create(String tableName, boolean limitVersion)
-      throws AccumuloException, AccumuloSecurityException, TableExistsException {
-    if (limitVersion) {
-      create(tableName);
-    } else {
-      create(tableName, new NewTableConfiguration().withoutDefaultIterators());
-    }
-  }
-
-  /**
-   * @param tableName the name of the table
-   * @param versioningIter Enables/disables the versioning iterator, which will limit the number of
-   *        Key versions kept.
-   * @param timeType specifies logical or real-time based time recording for entries in the table
-   * @throws AccumuloException if a general error occurs
-   * @throws AccumuloSecurityException if the user does not have permission
-   * @throws TableExistsException if the table already exists
-   * @deprecated since 1.7.0; use {@link #create(String, NewTableConfiguration)} instead.
-   */
-  @Deprecated(since = "1.7.0")
-  default void create(String tableName, boolean versioningIter, TimeType timeType)
-      throws AccumuloException, AccumuloSecurityException, TableExistsException {
-    NewTableConfiguration ntc = new NewTableConfiguration().setTimeType(timeType);
-
-    if (versioningIter) {
-      create(tableName, ntc);
-    } else {
-      create(tableName, ntc.withoutDefaultIterators());
-    }
-  }
-
-  /**
    * Create a table with specified configuration. A safe way to ignore tables that do exist would be
    * to do something like the following:
    *
@@ -228,21 +187,6 @@
    * @param tableName the name of the table
    * @return the split points (end-row names) for the table's current split profile
    * @throws TableNotFoundException if the table does not exist
-   * @deprecated since 1.5.0; use {@link #listSplits(String)} instead.
-   */
-  @Deprecated(since = "1.5.0")
-  default Collection<Text> getSplits(String tableName) throws TableNotFoundException {
-    try {
-      return listSplits(tableName);
-    } catch (AccumuloSecurityException | AccumuloException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  /**
-   * @param tableName the name of the table
-   * @return the split points (end-row names) for the table's current split profile
-   * @throws TableNotFoundException if the table does not exist
    * @throws AccumuloException if a general error occurs
    * @throws AccumuloSecurityException if the user does not have permission
    * @since 1.5.0
@@ -253,23 +197,6 @@
   /**
    * @param tableName the name of the table
    * @param maxSplits specifies the maximum number of splits to return
-   * @return the split points (end-row names) for the table's current split profile, grouped into
-   *         fewer splits so as not to exceed maxSplits
-   * @deprecated since 1.5.0; use {@link #listSplits(String, int)} instead.
-   */
-  @Deprecated(since = "1.5.0")
-  default Collection<Text> getSplits(String tableName, int maxSplits)
-      throws TableNotFoundException {
-    try {
-      return listSplits(tableName, maxSplits);
-    } catch (AccumuloSecurityException | AccumuloException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  /**
-   * @param tableName the name of the table
-   * @param maxSplits specifies the maximum number of splits to return
    * @throws AccumuloException if a general error occurs
    * @throws AccumuloSecurityException if the user does not have permission
    * @return the split points (end-row names) for the table's current split profile, grouped into
@@ -363,8 +290,8 @@
 
   /**
    * Starts a full major compaction of the tablets in the range (start, end]. If the config does not
-   * specify a compaction selector (or a deprecated strategy), then all files in a tablet are
-   * compacted. The compaction is performed even for tablets that have only one file.
+   * specify a compaction selector, then all files in a tablet are compacted. The compaction is
+   * performed even for tablets that have only one file.
    *
    * <p>
    * The following optional settings can only be set by one compact call per table at the same time.
@@ -374,8 +301,6 @@
    * <li>Selector : {@link CompactionConfig#setSelector(PluginConfig)}</li>
    * <li>Configurer : {@link CompactionConfig#setConfigurer(PluginConfig)}</li>
    * <li>Iterators : {@link CompactionConfig#setIterators(List)}</li>
-   * <li>Compaction strategy (deprecated) :
-   * {@code CompactionConfig.setCompactionStrategy(CompactionStrategyConfig)}</li>
    * </ul>
    *
    * <p>
@@ -640,27 +565,6 @@
       throws AccumuloException, AccumuloSecurityException, TableNotFoundException;
 
   /**
-   * Bulk import all the files in a directory into a table. Files can be created using
-   * {@link RFile#newWriter()}
-   *
-   * @param tableName the name of the table
-   * @param dir the HDFS directory to find files for importing
-   * @param failureDir the HDFS directory to place files that failed to be imported, must exist and
-   *        be empty
-   * @param setTime override the time values in the input files, and use the current time for all
-   *        mutations
-   * @throws IOException when there is an error reading/writing to HDFS
-   * @throws AccumuloException when there is a general accumulo error
-   * @throws AccumuloSecurityException when the user does not have the proper permissions
-   * @throws TableNotFoundException when the table no longer exists
-   *
-   * @deprecated since 2.0.0 use {@link #importDirectory(String)} instead.
-   */
-  @Deprecated(since = "2.0.0")
-  void importDirectory(String tableName, String dir, String failureDir, boolean setTime)
-      throws TableNotFoundException, IOException, AccumuloException, AccumuloSecurityException;
-
-  /**
    * @since 2.0.0
    */
   interface ImportOptions {
@@ -749,11 +653,9 @@
    * Bulk import the files in a directory into a table. Files can be created using
    * {@link RFile#newWriter()}.
    * <p>
-   * This new method of bulk import examines files in the current process outside of holding a table
-   * lock. The old bulk import method ({@link #importDirectory(String, String, String, boolean)})
-   * examines files on the server side while holding a table read lock.
-   * <p>
-   * This API supports adding files to online and offline tables.
+   * This API supports adding files to online and offline tables. The files are examined on the
+   * client side to determine destination tablets. This examination will use memory and cpu within
+   * the process calling this API.
    * <p>
    * For example, to bulk import files from the directory 'dir1' into the table 'table1' use the
    * following code.
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/compaction/CompactableFile.java b/core/src/main/java/org/apache/accumulo/core/client/admin/compaction/CompactableFile.java
index baf5f04..36b2d7b 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/compaction/CompactableFile.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/compaction/CompactableFile.java
@@ -20,6 +20,7 @@
 
 import java.net.URI;
 
+import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.metadata.CompactableFileImpl;
 
 /**
@@ -33,6 +34,16 @@
 
   public URI getUri();
 
+  /**
+   * @return A range associated with the file. If a file has an associated range then Accumulo will
+   *         limit reads to within the range. Not all files have an associated range, it a file does
+   *         not have a range then an infinite range is returned. The URI plus this range uniquely
+   *         identify a file.
+   *
+   * @since 3.1.0
+   */
+  public Range getRange();
+
   public long getEstimatedSize();
 
   public long getEstimatedEntries();
@@ -41,4 +52,12 @@
     return new CompactableFileImpl(uri, estimatedSize, estimatedEntries);
   }
 
+  /**
+   * Creates a new CompactableFile object that implements this interface.
+   *
+   * @since 3.1.0
+   */
+  static CompactableFile create(URI uri, Range range, long estimatedSize, long estimatedEntries) {
+    return new CompactableFileImpl(uri, range, estimatedSize, estimatedEntries);
+  }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/compaction/CompactionConfigurer.java b/core/src/main/java/org/apache/accumulo/core/client/admin/compaction/CompactionConfigurer.java
index 6a58704..1e0e47c 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/compaction/CompactionConfigurer.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/compaction/CompactionConfigurer.java
@@ -20,9 +20,11 @@
 
 import java.util.Collection;
 import java.util.Map;
+import java.util.Set;
 
 import org.apache.accumulo.core.client.PluginEnvironment;
 import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.data.TabletId;
 
 /**
  * Enables dynamically overriding of per table properties used to create the output file for a
@@ -50,8 +52,36 @@
   public interface InputParameters {
     TableId getTableId();
 
+    /**
+     * @return the id of the tablet being compacted
+     * @since 3.0.0
+     */
+    TabletId getTabletId();
+
     public Collection<CompactableFile> getInputFiles();
 
+    /**
+     * For user and selector compactions:
+     * <ul>
+     * <li>Returns the selected set of files to be compacted.</li>
+     * <li>When getInputFiles() (inputFiles) and getSelectedFiles() (selectedFiles) are equal, then
+     * this is the final compaction.</li>
+     * <li>When they are not equal, this is an intermediate compaction.</li>
+     * <li>Intermediate compactions are compactions whose resultant RFile will be consumed by
+     * another compaction.</li>
+     * <li>inputFiles and selectedFiles can be compared using: <code>
+     * selectedFiles.equals(inputFiles instanceof Set ? inputFiles : Set.copyOf(inputFiles))
+     * </code></li>
+     * </ul>
+     * For system compactions:
+     * <ul>
+     * <li>There is no selected set of files so the empty set is returned.</li>
+     * </ul>
+     *
+     * @since 3.1
+     */
+    public Set<CompactableFile> getSelectedFiles();
+
     PluginEnvironment getEnvironment();
   }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/compaction/CompactionSelector.java b/core/src/main/java/org/apache/accumulo/core/client/admin/compaction/CompactionSelector.java
index d54f612..713a451 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/compaction/CompactionSelector.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/compaction/CompactionSelector.java
@@ -30,12 +30,14 @@
 import org.apache.accumulo.core.client.summary.Summary;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.data.TabletId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 
 /**
  * This class selects which files a user compaction will compact. It can also be configured per
- * table to periodically select files to compact.
+ * table to periodically select files to compact, although per table functionality is deprecated.
+ * See {@link org.apache.accumulo.core.spi.compaction.CompactionKind#SELECTOR} for details.
  *
  * @since 2.1.0
  */
@@ -61,6 +63,12 @@
 
     TableId getTableId();
 
+    /**
+     * @return the tablet id of the tablet being compacted
+     * @since 3.0.0
+     */
+    TabletId getTabletId();
+
     Optional<SortedKeyValueIterator<Key,Value>> getSample(CompactableFile cf,
         SamplerConfiguration sc);
 
diff --git a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/BigIntegerLexicoder.java b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/BigIntegerLexicoder.java
index 1d468e4..9195684 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/BigIntegerLexicoder.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/BigIntegerLexicoder.java
@@ -22,6 +22,7 @@
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
+import java.io.UncheckedIOException;
 import java.math.BigInteger;
 
 import org.apache.accumulo.core.clientImpl.lexicoder.FixedByteArrayOutputStream;
@@ -59,7 +60,7 @@
 
       return ret;
     } catch (IOException ioe) {
-      throw new RuntimeException(ioe);
+      throw new UncheckedIOException(ioe);
     }
 
   }
@@ -87,7 +88,7 @@
 
       return new BigInteger(bytes);
     } catch (IOException ioe) {
-      throw new RuntimeException(ioe);
+      throw new UncheckedIOException(ioe);
     }
   }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/PairLexicoder.java b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/PairLexicoder.java
index 590c370..e5172a5 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/PairLexicoder.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/PairLexicoder.java
@@ -80,7 +80,7 @@
 
     byte[][] fields = split(data, offset, len);
     if (fields.length != 2) {
-      throw new RuntimeException("Data does not have 2 fields, it has " + fields.length);
+      throw new IllegalArgumentException("Data does not have 2 fields, it has " + fields.length);
     }
 
     return new ComparablePair<>(firstLexicoder.decode(unescape(fields[0])),
diff --git a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/UUIDLexicoder.java b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/UUIDLexicoder.java
index b3a7bc2..fa1f3fd 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/UUIDLexicoder.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/UUIDLexicoder.java
@@ -22,6 +22,7 @@
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
+import java.io.UncheckedIOException;
 import java.util.UUID;
 
 import org.apache.accumulo.core.clientImpl.lexicoder.FixedByteArrayOutputStream;
@@ -52,7 +53,7 @@
 
       return ret;
     } catch (IOException e) {
-      throw new RuntimeException(e);
+      throw new UncheckedIOException(e);
     }
   }
 
@@ -69,7 +70,7 @@
       DataInputStream in = new DataInputStream(new ByteArrayInputStream(b, offset, len));
       return new UUID(in.readLong() ^ 0x8000000000000000L, in.readLong() ^ 0x8000000000000000L);
     } catch (IOException e) {
-      throw new RuntimeException(e);
+      throw new UncheckedIOException(e);
     }
   }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java
deleted file mode 100644
index 088a065..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java
+++ /dev/null
@@ -1,753 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.client.mapred;
-
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
-
-import java.io.IOException;
-import java.net.InetAddress;
-import java.security.SecureRandom;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.BatchScanner;
-import org.apache.accumulo.core.client.ClientSideIteratorScanner;
-import org.apache.accumulo.core.client.IsolatedScanner;
-import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.ScannerBase;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.admin.DelegationTokenConfig;
-import org.apache.accumulo.core.client.admin.SecurityOperations;
-import org.apache.accumulo.core.client.sample.SamplerConfiguration;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.client.security.tokens.DelegationToken;
-import org.apache.accumulo.core.client.security.tokens.KerberosToken;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.clientImpl.AuthenticationTokenIdentifier;
-import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.clientImpl.DelegationTokenImpl;
-import org.apache.accumulo.core.clientImpl.OfflineScanner;
-import org.apache.accumulo.core.clientImpl.ScannerImpl;
-import org.apache.accumulo.core.clientImpl.TabletLocator;
-import org.apache.accumulo.core.clientImpl.mapreduce.lib.InputConfigurator;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.dataImpl.KeyExtent;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.Pair;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.InputFormat;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.RecordReader;
-import org.apache.hadoop.security.token.Token;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
-
-/**
- * An abstract input format to provide shared methods common to all other input format classes. At
- * the very least, any classes inheriting from this class will need to define their own
- * {@link RecordReader}.
- *
- * @deprecated since 2.0.0; Use org.apache.accumulo.hadoop.mapred instead from the
- *             accumulo-hadoop-mapreduce.jar
- */
-@Deprecated(since = "2.0.0")
-public abstract class AbstractInputFormat<K,V> implements InputFormat<K,V> {
-
-  private static final SecureRandom random = new SecureRandom();
-
-  protected static final Class<?> CLASS = AccumuloInputFormat.class;
-  protected static final Logger log = Logger.getLogger(CLASS);
-
-  /**
-   * Sets the name of the classloader context on this scanner
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param context name of the classloader context
-   * @since 1.8.0
-   */
-  public static void setClassLoaderContext(JobConf job, String context) {
-    InputConfigurator.setClassLoaderContext(CLASS, job, context);
-  }
-
-  /**
-   * Returns the name of the current classloader context set on this scanner
-   *
-   * @param job the Hadoop job instance to be configured
-   * @return name of the current context
-   * @since 1.8.0
-   */
-  public static String getClassLoaderContext(JobConf job) {
-    return InputConfigurator.getClassLoaderContext(CLASS, job);
-  }
-
-  /**
-   * Sets the connector information needed to communicate with Accumulo in this job.
-   *
-   * <p>
-   * <b>WARNING:</b> Some tokens, when serialized, divulge sensitive information in the
-   * configuration as a means to pass the token to MapReduce tasks. This information is BASE64
-   * encoded to provide a charset safe conversion to a string, but this conversion is not intended
-   * to be secure. {@link PasswordToken} is one example that is insecure in this way; however
-   * {@link DelegationToken}s, acquired using
-   * {@link SecurityOperations#getDelegationToken(DelegationTokenConfig)}, is not subject to this
-   * concern.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param principal a valid Accumulo user name (user must have Table.CREATE permission)
-   * @param token the user's password
-   * @since 1.5.0
-   */
-  public static void setConnectorInfo(JobConf job, String principal, AuthenticationToken token)
-      throws AccumuloSecurityException {
-    if (token instanceof KerberosToken) {
-      log.info("Received KerberosToken, attempting to fetch DelegationToken");
-      try {
-        ClientContext client = InputConfigurator.client(CLASS, job);
-        token = client.securityOperations().getDelegationToken(new DelegationTokenConfig());
-      } catch (Exception e) {
-        log.warn("Failed to automatically obtain DelegationToken, Mappers/Reducers will likely"
-            + " fail to communicate with Accumulo", e);
-      }
-    }
-    // DelegationTokens can be passed securely from user to task without serializing insecurely in
-    // the configuration
-    if (token instanceof DelegationTokenImpl) {
-      DelegationTokenImpl delegationToken = (DelegationTokenImpl) token;
-
-      // Convert it into a Hadoop Token
-      AuthenticationTokenIdentifier identifier = delegationToken.getIdentifier();
-      Token<AuthenticationTokenIdentifier> hadoopToken = new Token<>(identifier.getBytes(),
-          delegationToken.getPassword(), identifier.getKind(), delegationToken.getServiceName());
-
-      // Add the Hadoop Token to the Job so it gets serialized and passed along.
-      job.getCredentials().addToken(hadoopToken.getService(), hadoopToken);
-    }
-
-    InputConfigurator.setConnectorInfo(CLASS, job, principal, token);
-  }
-
-  /**
-   * Sets the connector information needed to communicate with Accumulo in this job.
-   *
-   * <p>
-   * Stores the password in a file in HDFS and pulls that into the Distributed Cache in an attempt
-   * to be more secure than storing it in the Configuration.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param principal a valid Accumulo user name (user must have Table.CREATE permission)
-   * @param tokenFile the path to the token file
-   * @since 1.6.0
-   */
-  public static void setConnectorInfo(JobConf job, String principal, String tokenFile)
-      throws AccumuloSecurityException {
-    InputConfigurator.setConnectorInfo(CLASS, job, principal, tokenFile);
-  }
-
-  /**
-   * Determines if the connector has been configured.
-   *
-   * @param job the Hadoop context for the configured job
-   * @return true if the connector has been configured, false otherwise
-   * @since 1.5.0
-   * @see #setConnectorInfo(JobConf, String, AuthenticationToken)
-   */
-  protected static Boolean isConnectorInfoSet(JobConf job) {
-    return InputConfigurator.isConnectorInfoSet(CLASS, job);
-  }
-
-  /**
-   * Gets the user name from the configuration.
-   *
-   * @param job the Hadoop context for the configured job
-   * @return the user name
-   * @since 1.5.0
-   * @see #setConnectorInfo(JobConf, String, AuthenticationToken)
-   */
-  protected static String getPrincipal(JobConf job) {
-    return InputConfigurator.getPrincipal(CLASS, job);
-  }
-
-  /**
-   * Gets the authenticated token from either the specified token file or directly from the
-   * configuration, whichever was used when the job was configured.
-   *
-   * @param job the Hadoop context for the configured job
-   * @return the principal's authentication token
-   * @since 1.6.0
-   * @see #setConnectorInfo(JobConf, String, AuthenticationToken)
-   * @see #setConnectorInfo(JobConf, String, String)
-   */
-  protected static AuthenticationToken getAuthenticationToken(JobConf job) {
-    AuthenticationToken token = InputConfigurator.getAuthenticationToken(CLASS, job);
-    return InputConfigurator.unwrapAuthenticationToken(job, token);
-  }
-
-  /**
-   * Configures a {@link org.apache.accumulo.core.client.ZooKeeperInstance} for this job.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param instanceName the Accumulo instance name
-   * @param zooKeepers a comma-separated list of zookeeper servers
-   * @since 1.5.0
-   * @deprecated since 1.6.0; Use
-   *             {@link #setZooKeeperInstance(JobConf, org.apache.accumulo.core.client.ClientConfiguration)}
-   *             instead.
-   */
-  @Deprecated(since = "1.6.0")
-  public static void setZooKeeperInstance(JobConf job, String instanceName, String zooKeepers) {
-    setZooKeeperInstance(job, org.apache.accumulo.core.client.ClientConfiguration.create()
-        .withInstance(instanceName).withZkHosts(zooKeepers));
-  }
-
-  /**
-   * Configures a {@link org.apache.accumulo.core.client.ZooKeeperInstance} for this job.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param clientConfig client configuration containing connection options
-   * @since 1.6.0
-   */
-  public static void setZooKeeperInstance(JobConf job,
-      org.apache.accumulo.core.client.ClientConfiguration clientConfig) {
-    InputConfigurator.setZooKeeperInstance(CLASS, job, clientConfig);
-  }
-
-  /**
-   * Initializes an Accumulo {@link org.apache.accumulo.core.client.Instance} based on the
-   * configuration.
-   *
-   * @param job the Hadoop context for the configured job
-   * @return an Accumulo instance
-   * @since 1.5.0
-   * @see #setZooKeeperInstance(JobConf, org.apache.accumulo.core.client.ClientConfiguration)
-   */
-  protected static org.apache.accumulo.core.client.Instance getInstance(JobConf job) {
-    return InputConfigurator.getInstance(CLASS, job);
-  }
-
-  /**
-   * Sets the log level for this job.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param level the logging level
-   * @since 1.5.0
-   */
-  public static void setLogLevel(JobConf job, Level level) {
-    InputConfigurator.setLogLevel(CLASS, job, level);
-  }
-
-  /**
-   * Gets the log level from this configuration.
-   *
-   * @param job the Hadoop context for the configured job
-   * @return the log level
-   * @since 1.5.0
-   * @see #setLogLevel(JobConf, Level)
-   */
-  protected static Level getLogLevel(JobConf job) {
-    return InputConfigurator.getLogLevel(CLASS, job);
-  }
-
-  /**
-   * Sets the {@link Authorizations} used to scan. Must be a subset of the user's authorization.
-   * Defaults to the empty set.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param auths the user's authorizations
-   * @since 1.5.0
-   */
-  public static void setScanAuthorizations(JobConf job, Authorizations auths) {
-    InputConfigurator.setScanAuthorizations(CLASS, job, auths);
-  }
-
-  /**
-   * Gets the authorizations to set for the scans from the configuration.
-   *
-   * @param job the Hadoop context for the configured job
-   * @return the Accumulo scan authorizations
-   * @since 1.5.0
-   * @see #setScanAuthorizations(JobConf, Authorizations)
-   */
-  protected static Authorizations getScanAuthorizations(JobConf job) {
-    return InputConfigurator.getScanAuthorizations(CLASS, job);
-  }
-
-  /**
-   * Fetch the client configuration from the job.
-   *
-   * @param job The job
-   * @return The client configuration for the job
-   * @since 1.7.0
-   */
-  protected static org.apache.accumulo.core.client.ClientConfiguration
-      getClientConfiguration(JobConf job) {
-    return InputConfigurator.getClientConfiguration(CLASS, job);
-  }
-
-  // InputFormat doesn't have the equivalent of OutputFormat's checkOutputSpecs(JobContext job)
-  /**
-   * Check whether a configuration is fully configured to be used with an Accumulo
-   * {@link InputFormat}.
-   *
-   * @param job the Hadoop context for the configured job
-   * @throws java.io.IOException if the context is improperly configured
-   * @since 1.5.0
-   */
-  protected static void validateOptions(JobConf job) throws IOException {
-    InputConfigurator.validatePermissions(CLASS, job);
-  }
-
-  /**
-   * Fetches all {@link org.apache.accumulo.core.client.mapreduce.InputTableConfig}s that have been
-   * set on the given Hadoop job.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @return the {@link org.apache.accumulo.core.client.mapreduce.InputTableConfig} objects set on
-   *         the job
-   * @since 1.6.0
-   */
-  public static Map<String,org.apache.accumulo.core.client.mapreduce.InputTableConfig>
-      getInputTableConfigs(JobConf job) {
-    return InputConfigurator.getInputTableConfigs(CLASS, job);
-  }
-
-  /**
-   * Fetches a {@link org.apache.accumulo.core.client.mapreduce.InputTableConfig} that has been set
-   * on the configuration for a specific table.
-   *
-   * <p>
-   * null is returned in the event that the table doesn't exist.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param tableName the table name for which to grab the config object
-   * @return the {@link org.apache.accumulo.core.client.mapreduce.InputTableConfig} for the given
-   *         table
-   * @since 1.6.0
-   */
-  public static org.apache.accumulo.core.client.mapreduce.InputTableConfig
-      getInputTableConfig(JobConf job, String tableName) {
-    return InputConfigurator.getInputTableConfig(CLASS, job, tableName);
-  }
-
-  /**
-   * An abstract base class to be used to create {@link RecordReader} instances that convert from
-   * Accumulo {@link Key}/{@link Value} pairs to the user's K/V types.
-   *
-   * Subclasses must implement {@link #next(Object, Object)} to update key and value, and also to
-   * update the following variables:
-   * <ul>
-   * <li>Key {@link #currentKey} (used for progress reporting)</li>
-   * <li>int {@link #numKeysRead} (used for progress reporting)</li>
-   * </ul>
-   */
-  protected abstract static class AbstractRecordReader<K,V> implements RecordReader<K,V> {
-    protected long numKeysRead;
-    protected Iterator<Map.Entry<Key,Value>> scannerIterator;
-    protected RangeInputSplit split;
-    private org.apache.accumulo.core.client.mapreduce.RangeInputSplit baseSplit;
-    protected ScannerBase scannerBase;
-
-    /**
-     * Extracts Iterators settings from the context to be used by RecordReader.
-     *
-     * @param job the Hadoop job configuration
-     * @param tableName the table name for which the scanner is configured
-     * @return List of iterator settings for given table
-     * @since 1.7.0
-     */
-    protected abstract List<IteratorSetting> jobIterators(JobConf job, String tableName);
-
-    /**
-     * Configures the iterators on a scanner for the given table name.
-     *
-     * @param job the Hadoop job configuration
-     * @param scanner the scanner for which to configure the iterators
-     * @param tableName the table name for which the scanner is configured
-     * @since 1.7.0
-     */
-    private void setupIterators(JobConf job, ScannerBase scanner, String tableName,
-        org.apache.accumulo.core.client.mapreduce.RangeInputSplit split) {
-      List<IteratorSetting> iterators = null;
-
-      if (split == null) {
-        iterators = jobIterators(job, tableName);
-      } else {
-        iterators = split.getIterators();
-        if (iterators == null) {
-          iterators = jobIterators(job, tableName);
-        }
-      }
-
-      for (IteratorSetting iterator : iterators) {
-        scanner.addScanIterator(iterator);
-      }
-    }
-
-    /**
-     * Configures the iterators on a scanner for the given table name.
-     *
-     * @param job the Hadoop job configuration
-     * @param scanner the scanner for which to configure the iterators
-     * @param tableName the table name for which the scanner is configured
-     * @since 1.6.0
-     * @deprecated since 1.7.0; Use {@link #jobIterators} instead.
-     */
-    @Deprecated(since = "1.7.0")
-    protected void setupIterators(JobConf job, Scanner scanner, String tableName,
-        RangeInputSplit split) {
-      setupIterators(job, (ScannerBase) scanner, tableName, split);
-    }
-
-    /**
-     * Initialize a scanner over the given input split using this task attempt configuration.
-     */
-    public void initialize(InputSplit inSplit, JobConf job) throws IOException {
-      baseSplit = (org.apache.accumulo.core.client.mapreduce.RangeInputSplit) inSplit;
-      log.debug("Initializing input split: " + baseSplit);
-
-      Authorizations authorizations = baseSplit.getAuths();
-      if (null == authorizations) {
-        authorizations = getScanAuthorizations(job);
-      }
-
-      String classLoaderContext = getClassLoaderContext(job);
-      String table = baseSplit.getTableName();
-
-      // in case the table name changed, we can still use the previous name for terms of
-      // configuration, but the scanner will use the table id resolved at job setup time
-      org.apache.accumulo.core.client.mapreduce.InputTableConfig tableConfig =
-          getInputTableConfig(job, baseSplit.getTableName());
-
-      ClientContext client = InputConfigurator.client(CLASS, baseSplit, job);
-
-      log.debug("Created client with user: " + client.whoami());
-      log.debug("Creating scanner for table: " + table);
-      log.debug("Authorizations are: " + authorizations);
-
-      if (baseSplit instanceof org.apache.accumulo.core.clientImpl.mapred.BatchInputSplit) {
-        BatchScanner scanner;
-        org.apache.accumulo.core.clientImpl.mapred.BatchInputSplit multiRangeSplit =
-            (org.apache.accumulo.core.clientImpl.mapred.BatchInputSplit) baseSplit;
-
-        try {
-          // Note: BatchScanner will use at most one thread per tablet, currently BatchInputSplit
-          // will not span tablets
-          int scanThreads = 1;
-          scanner =
-              client.createBatchScanner(baseSplit.getTableName(), authorizations, scanThreads);
-          setupIterators(job, scanner, baseSplit.getTableName(), baseSplit);
-          if (classLoaderContext != null) {
-            scanner.setClassLoaderContext(classLoaderContext);
-          }
-        } catch (Exception e) {
-          throw new IOException(e);
-        }
-
-        scanner.setRanges(multiRangeSplit.getRanges());
-        scannerBase = scanner;
-
-      } else if (baseSplit instanceof RangeInputSplit) {
-        split = (RangeInputSplit) baseSplit;
-        Boolean isOffline = baseSplit.isOffline();
-        if (isOffline == null) {
-          isOffline = tableConfig.isOfflineScan();
-        }
-
-        Boolean isIsolated = baseSplit.isIsolatedScan();
-        if (isIsolated == null) {
-          isIsolated = tableConfig.shouldUseIsolatedScanners();
-        }
-
-        Boolean usesLocalIterators = baseSplit.usesLocalIterators();
-        if (usesLocalIterators == null) {
-          usesLocalIterators = tableConfig.shouldUseLocalIterators();
-        }
-
-        Scanner scanner;
-
-        try {
-          if (isOffline) {
-            scanner =
-                new OfflineScanner(client, TableId.of(baseSplit.getTableId()), authorizations);
-          } else {
-            scanner = new ScannerImpl(client, TableId.of(baseSplit.getTableId()), authorizations);
-          }
-          if (isIsolated) {
-            log.info("Creating isolated scanner");
-            @SuppressWarnings("resource")
-            var wrapped = new IsolatedScanner(scanner);
-            scanner = wrapped;
-          }
-          if (usesLocalIterators) {
-            log.info("Using local iterators");
-            scanner = new ClientSideIteratorScanner(scanner);
-          }
-          setupIterators(job, scanner, baseSplit.getTableName(), baseSplit);
-        } catch (Exception e) {
-          throw new IOException(e);
-        }
-
-        scanner.setRange(baseSplit.getRange());
-        scannerBase = scanner;
-      } else {
-        throw new IllegalArgumentException("Can not initialize from " + baseSplit.getClass());
-      }
-
-      Collection<Pair<Text,Text>> columns = baseSplit.getFetchedColumns();
-      if (columns == null) {
-        columns = tableConfig.getFetchedColumns();
-      }
-
-      // setup a scanner within the bounds of this split
-      for (Pair<Text,Text> c : columns) {
-        if (c.getSecond() != null) {
-          log.debug("Fetching column " + c.getFirst() + ":" + c.getSecond());
-          scannerBase.fetchColumn(c.getFirst(), c.getSecond());
-        } else {
-          log.debug("Fetching column family " + c.getFirst());
-          scannerBase.fetchColumnFamily(c.getFirst());
-        }
-      }
-
-      SamplerConfiguration samplerConfig = baseSplit.getSamplerConfiguration();
-      if (samplerConfig == null) {
-        samplerConfig = tableConfig.getSamplerConfiguration();
-      }
-
-      if (samplerConfig != null) {
-        scannerBase.setSamplerConfiguration(samplerConfig);
-      }
-
-      scannerIterator = scannerBase.iterator();
-      numKeysRead = 0;
-    }
-
-    @Override
-    public void close() {
-      if (scannerBase != null) {
-        scannerBase.close();
-      }
-    }
-
-    @Override
-    public long getPos() throws IOException {
-      return numKeysRead;
-    }
-
-    @Override
-    public float getProgress() throws IOException {
-      if (numKeysRead > 0 && currentKey == null) {
-        return 1.0f;
-      }
-      return baseSplit.getProgress(currentKey);
-    }
-
-    protected Key currentKey = null;
-
-  }
-
-  Map<String,Map<KeyExtent,List<Range>>> binOfflineTable(JobConf job, TableId tableId,
-      List<Range> ranges)
-      throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
-    return InputConfigurator.binOffline(tableId, ranges, InputConfigurator.client(CLASS, job));
-  }
-
-  /**
-   * Gets the splits of the tables that have been set on the job by reading the metadata table for
-   * the specified ranges.
-   *
-   * @return the splits from the tables based on the ranges.
-   * @throws java.io.IOException if a table set on the job doesn't exist or an error occurs
-   *         initializing the tablet locator
-   */
-  @Override
-  public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException {
-    Level logLevel = getLogLevel(job);
-    log.setLevel(logLevel);
-    validateOptions(job);
-
-    LinkedList<InputSplit> splits = new LinkedList<>();
-    Map<String,org.apache.accumulo.core.client.mapreduce.InputTableConfig> tableConfigs =
-        getInputTableConfigs(job);
-
-    for (Map.Entry<String,
-        org.apache.accumulo.core.client.mapreduce.InputTableConfig> tableConfigEntry : tableConfigs
-            .entrySet()) {
-
-      String tableName = tableConfigEntry.getKey();
-      org.apache.accumulo.core.client.mapreduce.InputTableConfig tableConfig =
-          tableConfigEntry.getValue();
-
-      ClientContext client;
-      try {
-        client = InputConfigurator.client(CLASS, job);
-      } catch (AccumuloException | AccumuloSecurityException e) {
-        throw new IOException(e);
-      }
-
-      TableId tableId;
-      // resolve table name to id once, and use id from this point forward
-      try {
-        tableId = client.getTableId(tableName);
-      } catch (TableNotFoundException e) {
-        throw new IOException(e);
-      }
-
-      boolean batchScan = InputConfigurator.isBatchScan(CLASS, job);
-      boolean supportBatchScan = !(tableConfig.isOfflineScan()
-          || tableConfig.shouldUseIsolatedScanners() || tableConfig.shouldUseLocalIterators());
-      if (batchScan && !supportBatchScan) {
-        throw new IllegalArgumentException("BatchScanner optimization not available for offline"
-            + " scan, isolated, or local iterators");
-      }
-
-      boolean autoAdjust = tableConfig.shouldAutoAdjustRanges();
-      if (batchScan && !autoAdjust) {
-        throw new IllegalArgumentException(
-            "AutoAdjustRanges must be enabled when using BatchScanner optimization");
-      }
-
-      List<Range> ranges =
-          autoAdjust ? Range.mergeOverlapping(tableConfig.getRanges()) : tableConfig.getRanges();
-      if (ranges.isEmpty()) {
-        ranges = new ArrayList<>(1);
-        ranges.add(new Range());
-      }
-
-      // get the metadata information for these ranges
-      Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<>();
-      TabletLocator tl;
-      try {
-        if (tableConfig.isOfflineScan()) {
-          binnedRanges = binOfflineTable(job, tableId, ranges);
-          while (binnedRanges == null) {
-            // Some tablets were still online, try again
-            // sleep randomly between 100 and 200 ms
-            sleepUninterruptibly(100 + random.nextInt(100), TimeUnit.MILLISECONDS);
-            binnedRanges = binOfflineTable(job, tableId, ranges);
-          }
-        } else {
-          tl = TabletLocator.getLocator(client, tableId);
-          // its possible that the cache could contain complete, but old information about a
-          // tables tablets... so clear it
-          tl.invalidateCache();
-
-          while (!tl.binRanges(client, ranges, binnedRanges).isEmpty()) {
-            client.requireNotDeleted(tableId);
-            client.requireNotOffline(tableId, tableName);
-            binnedRanges.clear();
-            log.warn("Unable to locate bins for specified ranges. Retrying.");
-            // sleep randomly between 100 and 200 ms
-            sleepUninterruptibly(100 + random.nextInt(100), TimeUnit.MILLISECONDS);
-            tl.invalidateCache();
-          }
-        }
-      } catch (Exception e) {
-        throw new IOException(e);
-      }
-
-      // all of this code will add either range per each locations or split ranges and add
-      // range-location split
-      // Map from Range to Array of Locations, we only use this if we're don't split
-      HashMap<Range,ArrayList<String>> splitsToAdd = null;
-
-      if (!autoAdjust) {
-        splitsToAdd = new HashMap<>();
-      }
-
-      HashMap<String,String> hostNameCache = new HashMap<>();
-      for (Map.Entry<String,Map<KeyExtent,List<Range>>> tserverBin : binnedRanges.entrySet()) {
-        String ip = tserverBin.getKey().split(":", 2)[0];
-        String location = hostNameCache.get(ip);
-        if (location == null) {
-          InetAddress inetAddress = InetAddress.getByName(ip);
-          location = inetAddress.getCanonicalHostName();
-          hostNameCache.put(ip, location);
-        }
-        for (Map.Entry<KeyExtent,List<Range>> extentRanges : tserverBin.getValue().entrySet()) {
-          Range ke = extentRanges.getKey().toDataRange();
-          if (batchScan) {
-            // group ranges by tablet to be read by a BatchScanner
-            ArrayList<Range> clippedRanges = new ArrayList<>();
-            for (Range r : extentRanges.getValue()) {
-              clippedRanges.add(ke.clip(r));
-            }
-            org.apache.accumulo.core.clientImpl.mapred.BatchInputSplit split =
-                new org.apache.accumulo.core.clientImpl.mapred.BatchInputSplit(tableName, tableId,
-                    clippedRanges, new String[] {location});
-            org.apache.accumulo.core.clientImpl.mapreduce.SplitUtils.updateSplit(split, tableConfig,
-                logLevel);
-
-            splits.add(split);
-          } else {
-            // not grouping by tablet
-            for (Range r : extentRanges.getValue()) {
-              if (autoAdjust) {
-                // divide ranges into smaller ranges, based on the tablets
-                RangeInputSplit split = new RangeInputSplit(tableName, tableId.canonical(),
-                    ke.clip(r), new String[] {location});
-                org.apache.accumulo.core.clientImpl.mapreduce.SplitUtils.updateSplit(split,
-                    tableConfig, logLevel);
-                split.setOffline(tableConfig.isOfflineScan());
-                split.setIsolatedScan(tableConfig.shouldUseIsolatedScanners());
-                split.setUsesLocalIterators(tableConfig.shouldUseLocalIterators());
-                splits.add(split);
-              } else {
-                // don't divide ranges
-                ArrayList<String> locations = splitsToAdd.get(r);
-                if (locations == null) {
-                  locations = new ArrayList<>(1);
-                }
-                locations.add(location);
-                splitsToAdd.put(r, locations);
-              }
-            }
-          }
-        }
-      }
-
-      if (!autoAdjust) {
-        for (Map.Entry<Range,ArrayList<String>> entry : splitsToAdd.entrySet()) {
-          RangeInputSplit split = new RangeInputSplit(tableName, tableId.canonical(),
-              entry.getKey(), entry.getValue().toArray(new String[0]));
-          org.apache.accumulo.core.clientImpl.mapreduce.SplitUtils.updateSplit(split, tableConfig,
-              logLevel);
-          split.setOffline(tableConfig.isOfflineScan());
-          split.setIsolatedScan(tableConfig.shouldUseIsolatedScanners());
-          split.setUsesLocalIterators(tableConfig.shouldUseLocalIterators());
-
-          splits.add(split);
-        }
-      }
-    }
-    return splits.toArray(new InputSplit[splits.size()]);
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormat.java
deleted file mode 100644
index 75c11ad..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormat.java
+++ /dev/null
@@ -1,176 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.client.mapred;
-
-import java.io.IOException;
-
-import org.apache.accumulo.core.client.rfile.RFile;
-import org.apache.accumulo.core.client.rfile.RFileWriter;
-import org.apache.accumulo.core.client.sample.SamplerConfiguration;
-import org.apache.accumulo.core.clientImpl.mapreduce.lib.FileOutputConfigurator;
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Value;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapred.FileOutputFormat;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.RecordWriter;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.hadoop.util.Progressable;
-import org.apache.log4j.Logger;
-
-/**
- * This class allows MapReduce jobs to write output in the Accumulo data file format.<br>
- * Care should be taken to write only sorted data (sorted by {@link Key}), as this is an important
- * requirement of Accumulo data files.
- *
- * <p>
- * The output path to be created must be specified via
- * {@link AccumuloFileOutputFormat#setOutputPath(JobConf, Path)}. This is inherited from
- * {@link FileOutputFormat#setOutputPath(JobConf, Path)}. Other methods from
- * {@link FileOutputFormat} are not supported and may be ignored or cause failures. Using other
- * Hadoop configuration options that affect the behavior of the underlying files directly in the
- * Job's configuration may work, but are not directly supported at this time.
- *
- * @deprecated since 2.0.0; Use org.apache.accumulo.hadoop.mapred instead from the
- *             accumulo-hadoop-mapreduce.jar
- */
-@Deprecated(since = "2.0.0")
-public class AccumuloFileOutputFormat extends FileOutputFormat<Key,Value> {
-
-  private static final Class<?> CLASS = AccumuloFileOutputFormat.class;
-  protected static final Logger log = Logger.getLogger(CLASS);
-
-  /**
-   * Sets the compression type to use for data blocks. Specifying a compression may require
-   * additional libraries to be available to your Job.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param compressionType one of "none", "gz", "bzip2", "lzo", "lz4", "snappy", or "zstd"
-   * @since 1.5.0
-   */
-  public static void setCompressionType(JobConf job, String compressionType) {
-    FileOutputConfigurator.setCompressionType(CLASS, job, compressionType);
-  }
-
-  /**
-   * Sets the size for data blocks within each file.<br>
-   * Data blocks are a span of key/value pairs stored in the file that are compressed and indexed as
-   * a group.
-   *
-   * <p>
-   * Making this value smaller may increase seek performance, but at the cost of increasing the size
-   * of the indexes (which can also affect seek performance).
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param dataBlockSize the block size, in bytes
-   * @since 1.5.0
-   */
-  public static void setDataBlockSize(JobConf job, long dataBlockSize) {
-    FileOutputConfigurator.setDataBlockSize(CLASS, job, dataBlockSize);
-  }
-
-  /**
-   * Sets the size for file blocks in the file system; file blocks are managed, and replicated, by
-   * the underlying file system.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param fileBlockSize the block size, in bytes
-   * @since 1.5.0
-   */
-  public static void setFileBlockSize(JobConf job, long fileBlockSize) {
-    FileOutputConfigurator.setFileBlockSize(CLASS, job, fileBlockSize);
-  }
-
-  /**
-   * Sets the size for index blocks within each file; smaller blocks means a deeper index hierarchy
-   * within the file, while larger blocks mean a more shallow index hierarchy within the file. This
-   * can affect the performance of queries.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param indexBlockSize the block size, in bytes
-   * @since 1.5.0
-   */
-  public static void setIndexBlockSize(JobConf job, long indexBlockSize) {
-    FileOutputConfigurator.setIndexBlockSize(CLASS, job, indexBlockSize);
-  }
-
-  /**
-   * Sets the file system replication factor for the resulting file, overriding the file system
-   * default.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param replication the number of replicas for produced files
-   * @since 1.5.0
-   */
-  public static void setReplication(JobConf job, int replication) {
-    FileOutputConfigurator.setReplication(CLASS, job, replication);
-  }
-
-  /**
-   * Specify a sampler to be used when writing out data. This will result in the output file having
-   * sample data.
-   *
-   * @param job The Hadoop job instance to be configured
-   * @param samplerConfig The configuration for creating sample data in the output file.
-   * @since 1.8.0
-   */
-
-  public static void setSampler(JobConf job, SamplerConfiguration samplerConfig) {
-    FileOutputConfigurator.setSampler(CLASS, job, samplerConfig);
-  }
-
-  @Override
-  public RecordWriter<Key,Value> getRecordWriter(FileSystem ignored, JobConf job, String name,
-      Progressable progress) throws IOException {
-    // get the path of the temporary output file
-    final Configuration conf = job;
-    final AccumuloConfiguration acuConf =
-        FileOutputConfigurator.getAccumuloConfiguration(CLASS, job);
-
-    final String extension = acuConf.get(Property.TABLE_FILE_TYPE);
-    final Path file =
-        new Path(getWorkOutputPath(job), getUniqueName(job, "part") + "." + extension);
-    final int visCacheSize = FileOutputConfigurator.getVisibilityCacheSize(conf);
-
-    return new RecordWriter<>() {
-      RFileWriter out = null;
-
-      @Override
-      public void close(Reporter reporter) throws IOException {
-        if (out != null) {
-          out.close();
-        }
-      }
-
-      @Override
-      public void write(Key key, Value value) throws IOException {
-        if (out == null) {
-          out = RFile.newWriter().to(file.toString()).withFileSystem(file.getFileSystem(conf))
-              .withTableProperties(acuConf).withVisibilityCacheSize(visCacheSize).build();
-          out.startDefaultLocalityGroup();
-        }
-        out.append(key, value);
-      }
-    };
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormat.java
deleted file mode 100644
index 4e59225..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormat.java
+++ /dev/null
@@ -1,107 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.client.mapred;
-
-import java.io.IOException;
-import java.util.Map.Entry;
-
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.format.DefaultFormatter;
-import org.apache.hadoop.mapred.InputFormat;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.RecordReader;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.log4j.Level;
-
-/**
- * This class allows MapReduce jobs to use Accumulo as the source of data. This {@link InputFormat}
- * provides keys and values of type {@link Key} and {@link Value} to the Map function.
- *
- * The user must specify the following via static configurator methods:
- *
- * <ul>
- * <li>{@link AccumuloInputFormat#setConnectorInfo(JobConf, String, AuthenticationToken)}
- * <li>{@link AccumuloInputFormat#setConnectorInfo(JobConf, String, String)}
- * <li>{@link AccumuloInputFormat#setInputTableName(JobConf, String)}
- * <li>{@link AccumuloInputFormat#setScanAuthorizations(JobConf, Authorizations)}
- * <li>{@link AccumuloInputFormat#setZooKeeperInstance(JobConf, org.apache.accumulo.core.client.ClientConfiguration)}
- * </ul>
- *
- * Other static methods are optional.
- *
- * @deprecated since 2.0.0; Use org.apache.accumulo.hadoop.mapred instead from the
- *             accumulo-hadoop-mapreduce.jar
- */
-@Deprecated(since = "2.0.0")
-public class AccumuloInputFormat extends InputFormatBase<Key,Value> {
-
-  @Override
-  public RecordReader<Key,Value> getRecordReader(InputSplit split, JobConf job, Reporter reporter)
-      throws IOException {
-    log.setLevel(getLogLevel(job));
-
-    // Override the log level from the configuration as if the RangeInputSplit has one it's the more
-    // correct one to use.
-    if (split instanceof org.apache.accumulo.core.client.mapreduce.RangeInputSplit) {
-      org.apache.accumulo.core.client.mapreduce.RangeInputSplit accSplit =
-          (org.apache.accumulo.core.client.mapreduce.RangeInputSplit) split;
-      Level level = accSplit.getLogLevel();
-      if (level != null) {
-        log.setLevel(level);
-      }
-    } else {
-      throw new IllegalArgumentException("No RecordReader for " + split.getClass());
-    }
-
-    RecordReaderBase<Key,Value> recordReader = new RecordReaderBase<>() {
-
-      @Override
-      public boolean next(Key key, Value value) throws IOException {
-        if (scannerIterator.hasNext()) {
-          ++numKeysRead;
-          Entry<Key,Value> entry = scannerIterator.next();
-          key.set(currentKey = entry.getKey());
-          value.set(entry.getValue().get());
-          if (log.isTraceEnabled()) {
-            log.trace("Processing key/value pair: " + DefaultFormatter.formatEntry(entry, true));
-          }
-          return true;
-        }
-        return false;
-      }
-
-      @Override
-      public Key createKey() {
-        return new Key();
-      }
-
-      @Override
-      public Value createValue() {
-        return new Value();
-      }
-
-    };
-    recordReader.initialize(split, job);
-    return recordReader;
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloMultiTableInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloMultiTableInputFormat.java
deleted file mode 100644
index d0680d0..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloMultiTableInputFormat.java
+++ /dev/null
@@ -1,108 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.client.mapred;
-
-import java.io.IOException;
-import java.util.Map;
-
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.format.DefaultFormatter;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.RecordReader;
-import org.apache.hadoop.mapred.Reporter;
-
-/**
- * This class allows MapReduce jobs to use multiple Accumulo tables as the source of data. This
- * {@link org.apache.hadoop.mapred.InputFormat} provides keys and values of type {@link Key} and
- * {@link Value} to the Map function.
- *
- * The user must specify the following via static configurator methods:
- *
- * <ul>
- * <li>{@link AccumuloMultiTableInputFormat#setConnectorInfo(JobConf, String, AuthenticationToken)}
- * <li>{@link AccumuloMultiTableInputFormat#setConnectorInfo(JobConf, String, String)}
- * <li>{@link AccumuloMultiTableInputFormat#setScanAuthorizations(JobConf, Authorizations)}
- * <li>{@link AccumuloMultiTableInputFormat#setZooKeeperInstance(JobConf, org.apache.accumulo.core.client.ClientConfiguration)}
- * <li>{@link AccumuloMultiTableInputFormat#setInputTableConfigs(org.apache.hadoop.mapred.JobConf, java.util.Map)}
- * </ul>
- *
- * Other static methods are optional.
- *
- * @deprecated since 2.0.0; Use org.apache.accumulo.hadoop.mapred instead from the
- *             accumulo-hadoop-mapreduce.jar
- */
-@Deprecated(since = "2.0.0")
-public class AccumuloMultiTableInputFormat extends AbstractInputFormat<Key,Value> {
-
-  /**
-   * Sets the {@link org.apache.accumulo.core.client.mapreduce.InputTableConfig} objects on the
-   * given Hadoop configuration
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param configs the table query configs to be set on the configuration.
-   * @since 1.6.0
-   */
-  public static void setInputTableConfigs(JobConf job,
-      Map<String,org.apache.accumulo.core.client.mapreduce.InputTableConfig> configs) {
-    org.apache.accumulo.core.clientImpl.mapreduce.lib.InputConfigurator.setInputTableConfigs(CLASS,
-        job, configs);
-  }
-
-  @Override
-  public RecordReader<Key,Value> getRecordReader(InputSplit split, JobConf job, Reporter reporter)
-      throws IOException {
-    log.setLevel(getLogLevel(job));
-    InputFormatBase.RecordReaderBase<Key,Value> recordReader =
-        new InputFormatBase.RecordReaderBase<>() {
-
-          @Override
-          public boolean next(Key key, Value value) throws IOException {
-            if (scannerIterator.hasNext()) {
-              ++numKeysRead;
-              Map.Entry<Key,Value> entry = scannerIterator.next();
-              key.set(currentKey = entry.getKey());
-              value.set(entry.getValue().get());
-              if (log.isTraceEnabled()) {
-                log.trace(
-                    "Processing key/value pair: " + DefaultFormatter.formatEntry(entry, true));
-              }
-              return true;
-            }
-            return false;
-          }
-
-          @Override
-          public Key createKey() {
-            return new Key();
-          }
-
-          @Override
-          public Value createValue() {
-            return new Value();
-          }
-
-        };
-    recordReader.initialize(split, job);
-    return recordReader;
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java
deleted file mode 100644
index 6dbecae..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java
+++ /dev/null
@@ -1,577 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.client.mapred;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Set;
-
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.BatchWriterConfig;
-import org.apache.accumulo.core.client.MultiTableBatchWriter;
-import org.apache.accumulo.core.client.MutationsRejectedException;
-import org.apache.accumulo.core.client.TableExistsException;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.admin.DelegationTokenConfig;
-import org.apache.accumulo.core.client.admin.SecurityOperations;
-import org.apache.accumulo.core.client.security.SecurityErrorCode;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken.AuthenticationTokenSerializer;
-import org.apache.accumulo.core.client.security.tokens.DelegationToken;
-import org.apache.accumulo.core.client.security.tokens.KerberosToken;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.clientImpl.AuthenticationTokenIdentifier;
-import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.clientImpl.DelegationTokenImpl;
-import org.apache.accumulo.core.clientImpl.mapreduce.lib.OutputConfigurator;
-import org.apache.accumulo.core.data.ColumnUpdate;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.security.ColumnVisibility;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.OutputFormat;
-import org.apache.hadoop.mapred.RecordWriter;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.util.Progressable;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
-
-/**
- * This class allows MapReduce jobs to use Accumulo as the sink for data. This {@link OutputFormat}
- * accepts keys and values of type {@link Text} (for a table name) and {@link Mutation} from the Map
- * and Reduce functions.
- *
- * The user must specify the following via static configurator methods:
- *
- * <ul>
- * <li>{@link AccumuloOutputFormat#setConnectorInfo(JobConf, String, AuthenticationToken)}
- * <li>{@link AccumuloOutputFormat#setConnectorInfo(JobConf, String, String)}
- * <li>{@link AccumuloOutputFormat#setZooKeeperInstance(JobConf, org.apache.accumulo.core.client.ClientConfiguration)}
- * </ul>
- *
- * Other static methods are optional.
- *
- * @deprecated since 2.0.0; Use org.apache.accumulo.hadoop.mapred instead from the
- *             accumulo-hadoop-mapreduce.jar
- */
-@Deprecated(since = "2.0.0")
-public class AccumuloOutputFormat implements OutputFormat<Text,Mutation> {
-
-  private static final Class<?> CLASS = AccumuloOutputFormat.class;
-  protected static final Logger log = Logger.getLogger(CLASS);
-
-  /**
-   * Sets the connector information needed to communicate with Accumulo in this job.
-   *
-   * <p>
-   * <b>WARNING:</b> Some tokens, when serialized, divulge sensitive information in the
-   * configuration as a means to pass the token to MapReduce tasks. This information is BASE64
-   * encoded to provide a charset safe conversion to a string, but this conversion is not intended
-   * to be secure. {@link PasswordToken} is one example that is insecure in this way; however
-   * {@link DelegationToken}s, acquired using
-   * {@link SecurityOperations#getDelegationToken(DelegationTokenConfig)}, is not subject to this
-   * concern.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param principal a valid Accumulo user name (user must have Table.CREATE permission if
-   *        {@link #setCreateTables(JobConf, boolean)} is set to true)
-   * @param token the user's password
-   * @since 1.5.0
-   */
-  public static void setConnectorInfo(JobConf job, String principal, AuthenticationToken token)
-      throws AccumuloSecurityException {
-    if (token instanceof KerberosToken) {
-      log.info("Received KerberosToken, attempting to fetch DelegationToken");
-      try {
-        ClientContext client = OutputConfigurator.client(CLASS, job);
-        token = client.securityOperations().getDelegationToken(new DelegationTokenConfig());
-      } catch (Exception e) {
-        log.warn("Failed to automatically obtain DelegationToken, "
-            + "Mappers/Reducers will likely fail to communicate with Accumulo", e);
-      }
-    }
-    // DelegationTokens can be passed securely from user to task without serializing insecurely in
-    // the configuration
-    if (token instanceof DelegationTokenImpl) {
-      DelegationTokenImpl delegationToken = (DelegationTokenImpl) token;
-
-      // Convert it into a Hadoop Token
-      AuthenticationTokenIdentifier identifier = delegationToken.getIdentifier();
-      Token<AuthenticationTokenIdentifier> hadoopToken = new Token<>(identifier.getBytes(),
-          delegationToken.getPassword(), identifier.getKind(), delegationToken.getServiceName());
-
-      // Add the Hadoop Token to the Job so it gets serialized and passed along.
-      job.getCredentials().addToken(hadoopToken.getService(), hadoopToken);
-    }
-
-    OutputConfigurator.setConnectorInfo(CLASS, job, principal, token);
-  }
-
-  /**
-   * Sets the connector information needed to communicate with Accumulo in this job.
-   *
-   * <p>
-   * Stores the password in a file in HDFS and pulls that into the Distributed Cache in an attempt
-   * to be more secure than storing it in the Configuration.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param principal a valid Accumulo user name (user must have Table.CREATE permission if
-   *        {@link #setCreateTables(JobConf, boolean)} is set to true)
-   * @param tokenFile the path to the password file
-   * @since 1.6.0
-   */
-  public static void setConnectorInfo(JobConf job, String principal, String tokenFile)
-      throws AccumuloSecurityException {
-    OutputConfigurator.setConnectorInfo(CLASS, job, principal, tokenFile);
-  }
-
-  /**
-   * Determines if the connector has been configured.
-   *
-   * @param job the Hadoop context for the configured job
-   * @return true if the connector has been configured, false otherwise
-   * @since 1.5.0
-   * @see #setConnectorInfo(JobConf, String, AuthenticationToken)
-   */
-  protected static Boolean isConnectorInfoSet(JobConf job) {
-    return OutputConfigurator.isConnectorInfoSet(CLASS, job);
-  }
-
-  /**
-   * Gets the principal from the configuration.
-   *
-   * @param job the Hadoop context for the configured job
-   * @return the user name
-   * @since 1.5.0
-   * @see #setConnectorInfo(JobConf, String, AuthenticationToken)
-   */
-  protected static String getPrincipal(JobConf job) {
-    return OutputConfigurator.getPrincipal(CLASS, job);
-  }
-
-  /**
-   * Gets the serialized token class from either the configuration or the token file.
-   *
-   * @since 1.5.0
-   * @deprecated since 1.6.0; Use {@link #getAuthenticationToken(JobConf)} instead.
-   */
-  @Deprecated(since = "1.6.0")
-  protected static String getTokenClass(JobConf job) {
-    return getAuthenticationToken(job).getClass().getName();
-  }
-
-  /**
-   * Gets the serialized token from either the configuration or the token file.
-   *
-   * @since 1.5.0
-   * @deprecated since 1.6.0; Use {@link #getAuthenticationToken(JobConf)} instead.
-   */
-  @Deprecated(since = "1.6.0")
-  protected static byte[] getToken(JobConf job) {
-    return AuthenticationTokenSerializer.serialize(getAuthenticationToken(job));
-  }
-
-  /**
-   * Gets the authenticated token from either the specified token file or directly from the
-   * configuration, whichever was used when the job was configured.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @return the principal's authentication token
-   * @since 1.6.0
-   * @see #setConnectorInfo(JobConf, String, AuthenticationToken)
-   * @see #setConnectorInfo(JobConf, String, String)
-   */
-  protected static AuthenticationToken getAuthenticationToken(JobConf job) {
-    AuthenticationToken token = OutputConfigurator.getAuthenticationToken(CLASS, job);
-    return OutputConfigurator.unwrapAuthenticationToken(job, token);
-  }
-
-  /**
-   * Configures a {@link org.apache.accumulo.core.client.ZooKeeperInstance} for this job.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param instanceName the Accumulo instance name
-   * @param zooKeepers a comma-separated list of zookeeper servers
-   * @since 1.5.0
-   * @deprecated since 1.6.0; Use
-   *             {@link #setZooKeeperInstance(JobConf, org.apache.accumulo.core.client.ClientConfiguration)}
-   *             instead.
-   */
-  @Deprecated(since = "1.6.0")
-  public static void setZooKeeperInstance(JobConf job, String instanceName, String zooKeepers) {
-    setZooKeeperInstance(job, org.apache.accumulo.core.client.ClientConfiguration.create()
-        .withInstance(instanceName).withZkHosts(zooKeepers));
-  }
-
-  /**
-   * Configures a {@link org.apache.accumulo.core.client.ZooKeeperInstance} for this job.
-   *
-   * @param job the Hadoop job instance to be configured
-   *
-   * @param clientConfig client configuration for specifying connection timeouts, SSL connection
-   *        options, etc.
-   * @since 1.6.0
-   */
-  public static void setZooKeeperInstance(JobConf job,
-      org.apache.accumulo.core.client.ClientConfiguration clientConfig) {
-    OutputConfigurator.setZooKeeperInstance(CLASS, job, clientConfig);
-  }
-
-  /**
-   * Initializes an Accumulo {@link org.apache.accumulo.core.client.Instance} based on the
-   * configuration.
-   *
-   * @param job the Hadoop context for the configured job
-   * @return an Accumulo instance
-   * @since 1.5.0
-   * @see #setZooKeeperInstance(JobConf, org.apache.accumulo.core.client.ClientConfiguration)
-   */
-  protected static org.apache.accumulo.core.client.Instance getInstance(JobConf job) {
-    return OutputConfigurator.getInstance(CLASS, job);
-  }
-
-  /**
-   * Sets the log level for this job.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param level the logging level
-   * @since 1.5.0
-   */
-  public static void setLogLevel(JobConf job, Level level) {
-    OutputConfigurator.setLogLevel(CLASS, job, level);
-  }
-
-  /**
-   * Gets the log level from this configuration.
-   *
-   * @param job the Hadoop context for the configured job
-   * @return the log level
-   * @since 1.5.0
-   * @see #setLogLevel(JobConf, Level)
-   */
-  protected static Level getLogLevel(JobConf job) {
-    return OutputConfigurator.getLogLevel(CLASS, job);
-  }
-
-  /**
-   * Sets the default table name to use if one emits a null in place of a table name for a given
-   * mutation. Table names can only be alpha-numeric and underscores.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param tableName the table to use when the tablename is null in the write call
-   * @since 1.5.0
-   */
-  public static void setDefaultTableName(JobConf job, String tableName) {
-    OutputConfigurator.setDefaultTableName(CLASS, job, tableName);
-  }
-
-  /**
-   * Gets the default table name from the configuration.
-   *
-   * @param job the Hadoop context for the configured job
-   * @return the default table name
-   * @since 1.5.0
-   * @see #setDefaultTableName(JobConf, String)
-   */
-  protected static String getDefaultTableName(JobConf job) {
-    return OutputConfigurator.getDefaultTableName(CLASS, job);
-  }
-
-  /**
-   * Sets the configuration for for the job's {@link BatchWriter} instances. If not set, a new
-   * {@link BatchWriterConfig}, with sensible built-in defaults is used. Setting the configuration
-   * multiple times overwrites any previous configuration.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param bwConfig the configuration for the {@link BatchWriter}
-   * @since 1.5.0
-   */
-  public static void setBatchWriterOptions(JobConf job, BatchWriterConfig bwConfig) {
-    OutputConfigurator.setBatchWriterOptions(CLASS, job, bwConfig);
-  }
-
-  /**
-   * Gets the {@link BatchWriterConfig} settings.
-   *
-   * @param job the Hadoop context for the configured job
-   * @return the configuration object
-   * @since 1.5.0
-   * @see #setBatchWriterOptions(JobConf, BatchWriterConfig)
-   */
-  protected static BatchWriterConfig getBatchWriterOptions(JobConf job) {
-    return OutputConfigurator.getBatchWriterOptions(CLASS, job);
-  }
-
-  /**
-   * Sets the directive to create new tables, as necessary. Table names can only be alpha-numeric
-   * and underscores.
-   *
-   * <p>
-   * By default, this feature is <b>disabled</b>.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param enableFeature the feature is enabled if true, disabled otherwise
-   * @since 1.5.0
-   */
-  public static void setCreateTables(JobConf job, boolean enableFeature) {
-    OutputConfigurator.setCreateTables(CLASS, job, enableFeature);
-  }
-
-  /**
-   * Determines whether tables are permitted to be created as needed.
-   *
-   * @param job the Hadoop context for the configured job
-   * @return true if the feature is disabled, false otherwise
-   * @since 1.5.0
-   * @see #setCreateTables(JobConf, boolean)
-   */
-  protected static Boolean canCreateTables(JobConf job) {
-    return OutputConfigurator.canCreateTables(CLASS, job);
-  }
-
-  /**
-   * Sets the directive to use simulation mode for this job. In simulation mode, no output is
-   * produced. This is useful for testing.
-   *
-   * <p>
-   * By default, this feature is <b>disabled</b>.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param enableFeature the feature is enabled if true, disabled otherwise
-   * @since 1.5.0
-   */
-  public static void setSimulationMode(JobConf job, boolean enableFeature) {
-    OutputConfigurator.setSimulationMode(CLASS, job, enableFeature);
-  }
-
-  /**
-   * Determines whether this feature is enabled.
-   *
-   * @param job the Hadoop context for the configured job
-   * @return true if the feature is enabled, false otherwise
-   * @since 1.5.0
-   * @see #setSimulationMode(JobConf, boolean)
-   */
-  protected static Boolean getSimulationMode(JobConf job) {
-    return OutputConfigurator.getSimulationMode(CLASS, job);
-  }
-
-  /**
-   * A base class to be used to create {@link RecordWriter} instances that write to Accumulo.
-   */
-  protected static class AccumuloRecordWriter implements RecordWriter<Text,Mutation> {
-    private MultiTableBatchWriter mtbw = null;
-    private HashMap<Text,BatchWriter> bws = null;
-    private Text defaultTableName = null;
-
-    private boolean simulate = false;
-    private boolean createTables = false;
-
-    private long mutCount = 0;
-    private long valCount = 0;
-
-    private AccumuloClient client;
-
-    protected AccumuloRecordWriter(JobConf job)
-        throws AccumuloException, AccumuloSecurityException, IOException {
-      Level l = getLogLevel(job);
-      if (l != null) {
-        log.setLevel(getLogLevel(job));
-      }
-      this.simulate = getSimulationMode(job);
-      this.createTables = canCreateTables(job);
-
-      if (simulate) {
-        log.info("Simulating output only. No writes to tables will occur");
-      }
-
-      this.bws = new HashMap<>();
-
-      String tname = getDefaultTableName(job);
-      this.defaultTableName = (tname == null) ? null : new Text(tname);
-
-      if (!simulate) {
-        this.client = OutputConfigurator.client(CLASS, job);
-        mtbw = client.createMultiTableBatchWriter(getBatchWriterOptions(job));
-      }
-    }
-
-    /**
-     * Push a mutation into a table. If table is null, the defaultTable will be used. If
-     * {@link AccumuloOutputFormat#canCreateTables(JobConf)} is set, the table will be created if it
-     * does not exist. The table name must only contain alphanumerics and underscore.
-     */
-    @Override
-    public void write(Text table, Mutation mutation) throws IOException {
-      if (table == null || table.getLength() == 0) {
-        table = this.defaultTableName;
-      }
-
-      if (!simulate && table == null) {
-        throw new IOException("No table or default table specified. Try simulation mode next time");
-      }
-
-      ++mutCount;
-      valCount += mutation.size();
-      printMutation(table, mutation);
-
-      if (simulate) {
-        return;
-      }
-
-      if (!bws.containsKey(table)) {
-        try {
-          addTable(table);
-        } catch (final Exception e) {
-          log.error("Could not add table '" + table + "'", e);
-          throw new IOException(e);
-        }
-      }
-
-      try {
-        bws.get(table).addMutation(mutation);
-      } catch (MutationsRejectedException e) {
-        throw new IOException(e);
-      }
-    }
-
-    public void addTable(Text tableName) throws AccumuloException, AccumuloSecurityException {
-      if (simulate) {
-        log.info("Simulating adding table: " + tableName);
-        return;
-      }
-
-      log.debug("Adding table: " + tableName);
-      BatchWriter bw = null;
-      String table = tableName.toString();
-
-      if (createTables && !client.tableOperations().exists(table)) {
-        try {
-          client.tableOperations().create(table);
-        } catch (AccumuloSecurityException e) {
-          log.error("Accumulo security violation creating " + table, e);
-          throw e;
-        } catch (TableExistsException e) {
-          // Shouldn't happen
-        }
-      }
-
-      try {
-        bw = mtbw.getBatchWriter(table);
-      } catch (TableNotFoundException e) {
-        log.error("Accumulo table " + table + " doesn't exist and cannot be created.", e);
-        throw new AccumuloException(e);
-      } catch (AccumuloException | AccumuloSecurityException e) {
-        throw e;
-      }
-
-      if (bw != null) {
-        bws.put(tableName, bw);
-      }
-    }
-
-    private int printMutation(Text table, Mutation m) {
-      if (log.isTraceEnabled()) {
-        log.trace(String.format("Table %s row key: %s", table, hexDump(m.getRow())));
-        for (ColumnUpdate cu : m.getUpdates()) {
-          log.trace(String.format("Table %s column: %s:%s", table, hexDump(cu.getColumnFamily()),
-              hexDump(cu.getColumnQualifier())));
-          log.trace(String.format("Table %s security: %s", table,
-              new ColumnVisibility(cu.getColumnVisibility())));
-          log.trace(String.format("Table %s value: %s", table, hexDump(cu.getValue())));
-        }
-      }
-      return m.getUpdates().size();
-    }
-
-    private String hexDump(byte[] ba) {
-      StringBuilder sb = new StringBuilder();
-      for (byte b : ba) {
-        if ((b > 0x20) && (b < 0x7e)) {
-          sb.append((char) b);
-        } else {
-          sb.append(String.format("x%02x", b));
-        }
-      }
-      return sb.toString();
-    }
-
-    @Override
-    public void close(Reporter reporter) throws IOException {
-      log.debug("mutations written: " + mutCount + ", values written: " + valCount);
-      if (simulate) {
-        return;
-      }
-
-      try {
-        mtbw.close();
-      } catch (MutationsRejectedException e) {
-        if (!e.getSecurityErrorCodes().isEmpty()) {
-          var tables = new HashMap<String,Set<SecurityErrorCode>>();
-          e.getSecurityErrorCodes().forEach((tabletId, secSet) -> {
-            var tableId = tabletId.getTable().canonical();
-            tables.computeIfAbsent(tableId, p -> new HashSet<>()).addAll(secSet);
-          });
-          log.error("Not authorized to write to tables : " + tables);
-        }
-
-        if (!e.getConstraintViolationSummaries().isEmpty()) {
-          log.error("Constraint violations : " + e.getConstraintViolationSummaries().size());
-        }
-        throw new IOException(e);
-      } finally {
-        client.close();
-      }
-    }
-  }
-
-  @Override
-  public void checkOutputSpecs(FileSystem ignored, JobConf job) throws IOException {
-    if (!isConnectorInfoSet(job)) {
-      throw new IOException("Connector info has not been set.");
-    }
-    try {
-      AccumuloClient c = OutputConfigurator.client(CLASS, job);
-      String principal = getPrincipal(job);
-      AuthenticationToken token = getAuthenticationToken(job);
-      if (!c.securityOperations().authenticateUser(principal, token)) {
-        throw new IOException("Unable to authenticate user");
-      }
-    } catch (AccumuloException | AccumuloSecurityException e) {
-      throw new IOException(e);
-    }
-  }
-
-  @Override
-  public RecordWriter<Text,Mutation> getRecordWriter(FileSystem ignored, JobConf job, String name,
-      Progressable progress) throws IOException {
-    try {
-      return new AccumuloRecordWriter(job);
-    } catch (Exception e) {
-      throw new IOException(e);
-    }
-  }
-
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloRowInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloRowInputFormat.java
deleted file mode 100644
index e8fa3e8..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloRowInputFormat.java
+++ /dev/null
@@ -1,99 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.client.mapred;
-
-import java.io.IOException;
-import java.util.Map.Entry;
-
-import org.apache.accumulo.core.client.RowIterator;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.PeekingIterator;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.InputFormat;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.RecordReader;
-import org.apache.hadoop.mapred.Reporter;
-
-/**
- * This class allows MapReduce jobs to use Accumulo as the source of data. This {@link InputFormat}
- * provides row names as {@link Text} as keys, and a corresponding {@link PeekingIterator} as a
- * value, which in turn makes the {@link Key}/{@link Value} pairs for that row available to the Map
- * function.
- *
- * The user must specify the following via static configurator methods:
- *
- * <ul>
- * <li>{@link AccumuloRowInputFormat#setConnectorInfo(JobConf, String, AuthenticationToken)}
- * <li>{@link AccumuloRowInputFormat#setInputTableName(JobConf, String)}
- * <li>{@link AccumuloRowInputFormat#setScanAuthorizations(JobConf, Authorizations)}
- * <li>{@link AccumuloRowInputFormat#setZooKeeperInstance(JobConf, org.apache.accumulo.core.client.ClientConfiguration)}
- * </ul>
- *
- * Other static methods are optional.
- *
- * @deprecated since 2.0.0; Use org.apache.accumulo.hadoop.mapred instead from the
- *             accumulo-hadoop-mapreduce.jar
- */
-@Deprecated(since = "2.0.0")
-public class AccumuloRowInputFormat
-    extends InputFormatBase<Text,PeekingIterator<Entry<Key,Value>>> {
-  @Override
-  public RecordReader<Text,PeekingIterator<Entry<Key,Value>>> getRecordReader(InputSplit split,
-      JobConf job, Reporter reporter) throws IOException {
-    log.setLevel(getLogLevel(job));
-    RecordReaderBase<Text,PeekingIterator<Entry<Key,Value>>> recordReader =
-        new RecordReaderBase<>() {
-          RowIterator rowIterator;
-
-          @Override
-          public void initialize(InputSplit inSplit, JobConf job) throws IOException {
-            super.initialize(inSplit, job);
-            rowIterator = new RowIterator(scannerIterator);
-          }
-
-          @Override
-          public boolean next(Text key, PeekingIterator<Entry<Key,Value>> value)
-              throws IOException {
-            if (!rowIterator.hasNext()) {
-              return false;
-            }
-            value.initialize(rowIterator.next());
-            numKeysRead = rowIterator.getKVCount();
-            key.set((currentKey = value.peek().getKey()).getRow());
-            return true;
-          }
-
-          @Override
-          public Text createKey() {
-            return new Text();
-          }
-
-          @Override
-          public PeekingIterator<Entry<Key,Value>> createValue() {
-            return new PeekingIterator<>();
-          }
-        };
-    recordReader.initialize(split, job);
-    return recordReader;
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java
deleted file mode 100644
index 597c38b..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java
+++ /dev/null
@@ -1,387 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.client.mapred;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.accumulo.core.client.BatchScanner;
-import org.apache.accumulo.core.client.ClientSideIteratorScanner;
-import org.apache.accumulo.core.client.IsolatedScanner;
-import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.ScannerBase;
-import org.apache.accumulo.core.client.admin.TableOperations;
-import org.apache.accumulo.core.client.sample.SamplerConfiguration;
-import org.apache.accumulo.core.clientImpl.mapreduce.lib.InputConfigurator;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.util.Pair;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.InputFormat;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.RecordReader;
-import org.apache.hadoop.mapred.Reporter;
-
-/**
- * This abstract {@link InputFormat} class allows MapReduce jobs to use Accumulo as the source of
- * K,V pairs.
- * <p>
- * Subclasses must implement a {@link #getRecordReader(InputSplit, JobConf, Reporter)} to provide a
- * {@link RecordReader} for K,V.
- * <p>
- * A static base class, RecordReaderBase, is provided to retrieve Accumulo {@link Key}/{@link Value}
- * pairs, but one must implement its {@link RecordReaderBase#next(Object, Object)} to transform them
- * to the desired generic types K,V.
- * <p>
- * See {@link AccumuloInputFormat} for an example implementation.
- *
- * @deprecated since 2.0.0; Use org.apache.accumulo.hadoop.mapred instead from the
- *             accumulo-hadoop-mapreduce.jar
- */
-@Deprecated(since = "2.0.0")
-public abstract class InputFormatBase<K,V> extends AbstractInputFormat<K,V> {
-
-  /**
-   * Sets the name of the input table, over which this job will scan.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param tableName the table to use when the tablename is null in the write call
-   * @since 1.5.0
-   */
-  public static void setInputTableName(JobConf job, String tableName) {
-    InputConfigurator.setInputTableName(CLASS, job, tableName);
-  }
-
-  /**
-   * Gets the table name from the configuration.
-   *
-   * @param job the Hadoop context for the configured job
-   * @return the table name
-   * @since 1.5.0
-   * @see #setInputTableName(JobConf, String)
-   */
-  protected static String getInputTableName(JobConf job) {
-    return InputConfigurator.getInputTableName(CLASS, job);
-  }
-
-  /**
-   * Sets the input ranges to scan for this job. If not set, the entire table will be scanned.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param ranges the ranges that will be mapped over
-   * @see TableOperations#splitRangeByTablets(String, Range, int)
-   * @since 1.5.0
-   */
-  public static void setRanges(JobConf job, Collection<Range> ranges) {
-    InputConfigurator.setRanges(CLASS, job, ranges);
-  }
-
-  /**
-   * Gets the ranges to scan over from a job.
-   *
-   * @param job the Hadoop context for the configured job
-   * @return the ranges
-   * @throws IOException if the ranges have been encoded improperly
-   * @since 1.5.0
-   * @see #setRanges(JobConf, Collection)
-   */
-  protected static List<Range> getRanges(JobConf job) throws IOException {
-    return InputConfigurator.getRanges(CLASS, job);
-  }
-
-  /**
-   * Restricts the columns that will be mapped over for this job.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param columnFamilyColumnQualifierPairs a pair of {@link Text} objects corresponding to column
-   *        family and column qualifier. If the column qualifier is null, the entire column family
-   *        is selected. An empty set is the default and is equivalent to scanning the all columns.
-   * @since 1.5.0
-   */
-  public static void fetchColumns(JobConf job,
-      Collection<Pair<Text,Text>> columnFamilyColumnQualifierPairs) {
-    InputConfigurator.fetchColumns(CLASS, job, columnFamilyColumnQualifierPairs);
-  }
-
-  /**
-   * Gets the columns to be mapped over from this job.
-   *
-   * @param job the Hadoop context for the configured job
-   * @return a set of columns
-   * @since 1.5.0
-   * @see #fetchColumns(JobConf, Collection)
-   */
-  protected static Set<Pair<Text,Text>> getFetchedColumns(JobConf job) {
-    return InputConfigurator.getFetchedColumns(CLASS, job);
-  }
-
-  /**
-   * Encode an iterator on the input for this job.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param cfg the configuration of the iterator
-   * @since 1.5.0
-   */
-  public static void addIterator(JobConf job, IteratorSetting cfg) {
-    InputConfigurator.addIterator(CLASS, job, cfg);
-  }
-
-  /**
-   * Gets a list of the iterator settings (for iterators to apply to a scanner) from this
-   * configuration.
-   *
-   * @param job the Hadoop context for the configured job
-   * @return a list of iterators
-   * @since 1.5.0
-   * @see #addIterator(JobConf, IteratorSetting)
-   */
-  protected static List<IteratorSetting> getIterators(JobConf job) {
-    return InputConfigurator.getIterators(CLASS, job);
-  }
-
-  /**
-   * Controls the automatic adjustment of ranges for this job. This feature merges overlapping
-   * ranges, then splits them to align with tablet boundaries. Disabling this feature will cause
-   * exactly one Map task to be created for each specified range. The default setting is enabled. *
-   *
-   * <p>
-   * By default, this feature is <b>enabled</b>.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param enableFeature the feature is enabled if true, disabled otherwise
-   * @see #setRanges(JobConf, Collection)
-   * @since 1.5.0
-   */
-  public static void setAutoAdjustRanges(JobConf job, boolean enableFeature) {
-    InputConfigurator.setAutoAdjustRanges(CLASS, job, enableFeature);
-  }
-
-  /**
-   * Determines whether a configuration has auto-adjust ranges enabled. Must be enabled when
-   * {@link #setBatchScan(JobConf, boolean)} is true.
-   *
-   * @param job the Hadoop context for the configured job
-   * @return false if the feature is disabled, true otherwise
-   * @since 1.5.0
-   * @see #setAutoAdjustRanges(JobConf, boolean)
-   */
-  protected static boolean getAutoAdjustRanges(JobConf job) {
-    return InputConfigurator.getAutoAdjustRanges(CLASS, job);
-  }
-
-  /**
-   * Controls the use of the {@link IsolatedScanner} in this job.
-   *
-   * <p>
-   * By default, this feature is <b>disabled</b>.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param enableFeature the feature is enabled if true, disabled otherwise
-   * @since 1.5.0
-   */
-  public static void setScanIsolation(JobConf job, boolean enableFeature) {
-    InputConfigurator.setScanIsolation(CLASS, job, enableFeature);
-  }
-
-  /**
-   * Determines whether a configuration has isolation enabled.
-   *
-   * @param job the Hadoop context for the configured job
-   * @return true if the feature is enabled, false otherwise
-   * @since 1.5.0
-   * @see #setScanIsolation(JobConf, boolean)
-   */
-  protected static boolean isIsolated(JobConf job) {
-    return InputConfigurator.isIsolated(CLASS, job);
-  }
-
-  /**
-   * Controls the use of the {@link ClientSideIteratorScanner} in this job. Enabling this feature
-   * will cause the iterator stack to be constructed within the Map task, rather than within the
-   * Accumulo TServer. To use this feature, all classes needed for those iterators must be available
-   * on the classpath for the task.
-   *
-   * <p>
-   * By default, this feature is <b>disabled</b>.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param enableFeature the feature is enabled if true, disabled otherwise
-   * @since 1.5.0
-   */
-  public static void setLocalIterators(JobConf job, boolean enableFeature) {
-    InputConfigurator.setLocalIterators(CLASS, job, enableFeature);
-  }
-
-  /**
-   * Determines whether a configuration uses local iterators.
-   *
-   * @param job the Hadoop context for the configured job
-   * @return true if the feature is enabled, false otherwise
-   * @since 1.5.0
-   * @see #setLocalIterators(JobConf, boolean)
-   */
-  protected static boolean usesLocalIterators(JobConf job) {
-    return InputConfigurator.usesLocalIterators(CLASS, job);
-  }
-
-  /**
-   * Enable reading offline tables. By default, this feature is disabled and only online tables are
-   * scanned. This will make the map reduce job directly read the table's files. If the table is not
-   * offline, then the job will fail. If the table comes online during the map reduce job, it is
-   * likely that the job will fail.
-   *
-   * <p>
-   * To use this option, the map reduce user will need access to read the Accumulo directory in
-   * HDFS.
-   *
-   * <p>
-   * Reading the offline table will create the scan time iterator stack in the map process. So any
-   * iterators that are configured for the table will need to be on the mapper's classpath.
-   *
-   * <p>
-   * One way to use this feature is to clone a table, take the clone offline, and use the clone as
-   * the input table for a map reduce job. If you plan to map reduce over the data many times, it
-   * may be better to the compact the table, clone it, take it offline, and use the clone for all
-   * map reduce jobs. The reason to do this is that compaction will reduce each tablet in the table
-   * to one file, and it is faster to read from one file.
-   *
-   * <p>
-   * There are two possible advantages to reading a tables file directly out of HDFS. First, you may
-   * see better read performance. Second, it will support speculative execution better. When reading
-   * an online table speculative execution can put more load on an already slow tablet server.
-   *
-   * <p>
-   * By default, this feature is <b>disabled</b>.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param enableFeature the feature is enabled if true, disabled otherwise
-   * @since 1.5.0
-   */
-  public static void setOfflineTableScan(JobConf job, boolean enableFeature) {
-    InputConfigurator.setOfflineTableScan(CLASS, job, enableFeature);
-  }
-
-  /**
-   * Determines whether a configuration has the offline table scan feature enabled.
-   *
-   * @param job the Hadoop context for the configured job
-   * @return true if the feature is enabled, false otherwise
-   * @since 1.5.0
-   * @see #setOfflineTableScan(JobConf, boolean)
-   */
-  protected static boolean isOfflineScan(JobConf job) {
-    return InputConfigurator.isOfflineScan(CLASS, job);
-  }
-
-  /**
-   * Controls the use of the {@link BatchScanner} in this job. Using this feature will group Ranges
-   * by their source tablet, producing an InputSplit per tablet rather than per Range. This batching
-   * helps to reduce overhead when querying a large number of small ranges. (ex: when doing
-   * quad-tree decomposition for spatial queries)
-   * <p>
-   * In order to achieve good locality of InputSplits this option always clips the input Ranges to
-   * tablet boundaries. This may result in one input Range contributing to several InputSplits.
-   * <p>
-   * Note: that the value of {@link #setAutoAdjustRanges(JobConf, boolean)} is ignored and is
-   * assumed to be true when BatchScan option is enabled.
-   * <p>
-   * This configuration is incompatible with:
-   * <ul>
-   * <li>{@link #setOfflineTableScan(JobConf, boolean)}</li>
-   * <li>{@link #setLocalIterators(JobConf, boolean)}</li>
-   * <li>{@link #setScanIsolation(JobConf, boolean)}</li>
-   * </ul>
-   * <p>
-   * By default, this feature is <b>disabled</b>.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param enableFeature the feature is enabled if true, disabled otherwise
-   * @since 1.7.0
-   */
-  public static void setBatchScan(JobConf job, boolean enableFeature) {
-    InputConfigurator.setBatchScan(CLASS, job, enableFeature);
-  }
-
-  /**
-   * Determines whether a configuration has the {@link BatchScanner} feature enabled.
-   *
-   * @param job the Hadoop context for the configured job
-   * @since 1.7.0
-   * @see #setBatchScan(JobConf, boolean)
-   */
-  public static boolean isBatchScan(JobConf job) {
-    return InputConfigurator.isBatchScan(CLASS, job);
-  }
-
-  /**
-   * Causes input format to read sample data. If sample data was created using a different
-   * configuration or a tables sampler configuration changes while reading data, then the input
-   * format will throw an error.
-   *
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param samplerConfig The sampler configuration that sample must have been created with inorder
-   *        for reading sample data to succeed.
-   *
-   * @since 1.8.0
-   * @see ScannerBase#setSamplerConfiguration(SamplerConfiguration)
-   */
-  public static void setSamplerConfiguration(JobConf job, SamplerConfiguration samplerConfig) {
-    InputConfigurator.setSamplerConfiguration(CLASS, job, samplerConfig);
-  }
-
-  protected abstract static class RecordReaderBase<K,V> extends AbstractRecordReader<K,V> {
-
-    @Override
-    @Deprecated(since = "2.0.0")
-    protected List<IteratorSetting> jobIterators(JobConf job, String tableName) {
-      return getIterators(job);
-    }
-
-    /**
-     * Apply the configured iterators to the scanner.
-     *
-     * @param iterators the iterators to set
-     * @param scanner the scanner to configure
-     * @deprecated since 1.7.0; Use {@link #jobIterators} instead.
-     */
-    @Deprecated(since = "1.7.0")
-    protected void setupIterators(List<IteratorSetting> iterators, Scanner scanner) {
-      for (IteratorSetting iterator : iterators) {
-        scanner.addScanIterator(iterator);
-      }
-    }
-
-    /**
-     * Apply the configured iterators from the configuration to the scanner.
-     *
-     * @param job the job configuration
-     * @param scanner the scanner to configure
-     */
-    @Deprecated(since = "1.7.0")
-    protected void setupIterators(JobConf job, Scanner scanner) {
-      setupIterators(getIterators(job), scanner);
-    }
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/RangeInputSplit.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/RangeInputSplit.java
deleted file mode 100644
index 3c1e749..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/RangeInputSplit.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.client.mapred;
-
-import java.io.IOException;
-
-import org.apache.accumulo.core.data.Range;
-import org.apache.hadoop.mapred.InputSplit;
-
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
-/**
- * The Class RangeInputSplit. Encapsulates an Accumulo range for use in Map Reduce jobs.
- *
- * @deprecated since 2.0.0; Use org.apache.accumulo.hadoop.mapred instead from the
- *             accumulo-hadoop-mapreduce.jar
- */
-@Deprecated(since = "2.0.0")
-@SuppressFBWarnings(value = "NM_SAME_SIMPLE_NAME_AS_SUPERCLASS",
-    justification = "Intended to share code between mapred and mapreduce")
-public class RangeInputSplit extends org.apache.accumulo.core.client.mapreduce.RangeInputSplit
-    implements InputSplit {
-
-  public RangeInputSplit() {}
-
-  public RangeInputSplit(RangeInputSplit split) throws IOException {
-    super(split);
-  }
-
-  protected RangeInputSplit(String table, String tableId, Range range, String[] locations) {
-    super(table, tableId, range, locations);
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
deleted file mode 100644
index 4324713..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
+++ /dev/null
@@ -1,783 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.client.mapreduce;
-
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
-
-import java.io.IOException;
-import java.net.InetAddress;
-import java.security.SecureRandom;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.BatchScanner;
-import org.apache.accumulo.core.client.ClientSideIteratorScanner;
-import org.apache.accumulo.core.client.IsolatedScanner;
-import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.ScannerBase;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.admin.DelegationTokenConfig;
-import org.apache.accumulo.core.client.admin.SecurityOperations;
-import org.apache.accumulo.core.client.sample.SamplerConfiguration;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.client.security.tokens.DelegationToken;
-import org.apache.accumulo.core.client.security.tokens.KerberosToken;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.clientImpl.AuthenticationTokenIdentifier;
-import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.clientImpl.DelegationTokenImpl;
-import org.apache.accumulo.core.clientImpl.OfflineScanner;
-import org.apache.accumulo.core.clientImpl.ScannerImpl;
-import org.apache.accumulo.core.clientImpl.TabletLocator;
-import org.apache.accumulo.core.clientImpl.mapreduce.lib.InputConfigurator;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.dataImpl.KeyExtent;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.Pair;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.security.token.Token;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
-
-/**
- * An abstract input format to provide shared methods common to all other input format classes. At
- * the very least, any classes inheriting from this class will need to define their own
- * {@link RecordReader}.
- *
- * @deprecated since 2.0.0; Use org.apache.accumulo.hadoop.mapreduce instead from the
- *             accumulo-hadoop-mapreduce.jar
- */
-@Deprecated(since = "2.0.0")
-public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
-
-  private static final SecureRandom random = new SecureRandom();
-
-  protected static final Class<?> CLASS = AccumuloInputFormat.class;
-  protected static final Logger log = Logger.getLogger(CLASS);
-
-  /**
-   * Sets the name of the classloader context on this scanner
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param context name of the classloader context
-   * @since 1.8.0
-   */
-  public static void setClassLoaderContext(Job job, String context) {
-    InputConfigurator.setClassLoaderContext(CLASS, job.getConfiguration(), context);
-  }
-
-  /**
-   * Returns the name of the current classloader context set on this scanner
-   *
-   * @param job the Hadoop job instance to be configured
-   * @return name of the current context
-   * @since 1.8.0
-   */
-  public static String getClassLoaderContext(JobContext job) {
-    return InputConfigurator.getClassLoaderContext(CLASS, job.getConfiguration());
-  }
-
-  /**
-   * Sets the connector information needed to communicate with Accumulo in this job.
-   *
-   * <p>
-   * <b>WARNING:</b> Some tokens, when serialized, divulge sensitive information in the
-   * configuration as a means to pass the token to MapReduce tasks. This information is BASE64
-   * encoded to provide a charset safe conversion to a string, but this conversion is not intended
-   * to be secure. {@link PasswordToken} is one example that is insecure in this way; however
-   * {@link DelegationToken}s, acquired using
-   * {@link SecurityOperations#getDelegationToken(DelegationTokenConfig)}, is not subject to this
-   * concern.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param principal a valid Accumulo user name (user must have Table.CREATE permission)
-   * @param token the user's password
-   * @since 1.5.0
-   */
-  public static void setConnectorInfo(Job job, String principal, AuthenticationToken token)
-      throws AccumuloSecurityException {
-    if (token instanceof KerberosToken) {
-      log.info("Received KerberosToken, attempting to fetch DelegationToken");
-      try {
-        ClientContext client = InputConfigurator.client(CLASS, job.getConfiguration());
-        token = client.securityOperations().getDelegationToken(new DelegationTokenConfig());
-      } catch (Exception e) {
-        log.warn("Failed to automatically obtain DelegationToken, "
-            + "Mappers/Reducers will likely fail to communicate with Accumulo", e);
-      }
-    }
-    // DelegationTokens can be passed securely from user to task without serializing insecurely in
-    // the configuration
-    if (token instanceof DelegationTokenImpl) {
-      DelegationTokenImpl delegationToken = (DelegationTokenImpl) token;
-
-      // Convert it into a Hadoop Token
-      AuthenticationTokenIdentifier identifier = delegationToken.getIdentifier();
-      Token<AuthenticationTokenIdentifier> hadoopToken = new Token<>(identifier.getBytes(),
-          delegationToken.getPassword(), identifier.getKind(), delegationToken.getServiceName());
-
-      // Add the Hadoop Token to the Job so it gets serialized and passed along.
-      job.getCredentials().addToken(hadoopToken.getService(), hadoopToken);
-    }
-
-    InputConfigurator.setConnectorInfo(CLASS, job.getConfiguration(), principal, token);
-  }
-
-  /**
-   * Sets the connector information needed to communicate with Accumulo in this job.
-   *
-   * <p>
-   * Stores the password in a file in HDFS and pulls that into the Distributed Cache in an attempt
-   * to be more secure than storing it in the Configuration.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param principal a valid Accumulo user name (user must have Table.CREATE permission)
-   * @param tokenFile the path to the token file
-   * @since 1.6.0
-   */
-  public static void setConnectorInfo(Job job, String principal, String tokenFile)
-      throws AccumuloSecurityException {
-    InputConfigurator.setConnectorInfo(CLASS, job.getConfiguration(), principal, tokenFile);
-  }
-
-  /**
-   * Determines if the connector has been configured.
-   *
-   * @param job the Hadoop context for the configured job
-   * @return true if the connector has been configured, false otherwise
-   * @since 1.5.0
-   * @see #setConnectorInfo(Job, String, AuthenticationToken)
-   */
-  protected static Boolean isConnectorInfoSet(JobContext job) {
-    return InputConfigurator.isConnectorInfoSet(CLASS, job.getConfiguration());
-  }
-
-  /**
-   * Gets the user name from the configuration.
-   *
-   * @param job the Hadoop context for the configured job
-   * @return the user name
-   * @since 1.5.0
-   * @see #setConnectorInfo(Job, String, AuthenticationToken)
-   */
-  protected static String getPrincipal(JobContext job) {
-    return InputConfigurator.getPrincipal(CLASS, job.getConfiguration());
-  }
-
-  /**
-   * Gets the serialized token class from either the configuration or the token file.
-   *
-   * @since 1.5.0
-   * @deprecated since 1.6.0; Use {@link #getAuthenticationToken(JobContext)} instead.
-   */
-  @Deprecated(since = "1.6.0")
-  protected static String getTokenClass(JobContext job) {
-    return getAuthenticationToken(job).getClass().getName();
-  }
-
-  /**
-   * Gets the serialized token from either the configuration or the token file.
-   *
-   * @since 1.5.0
-   * @deprecated since 1.6.0; Use {@link #getAuthenticationToken(JobContext)} instead.
-   */
-  @Deprecated(since = "1.6.0")
-  protected static byte[] getToken(JobContext job) {
-    return AuthenticationToken.AuthenticationTokenSerializer.serialize(getAuthenticationToken(job));
-  }
-
-  /**
-   * Gets the authenticated token from either the specified token file or directly from the
-   * configuration, whichever was used when the job was configured.
-   *
-   * @param job the Hadoop context for the configured job
-   * @return the principal's authentication token
-   * @since 1.6.0
-   * @see #setConnectorInfo(Job, String, AuthenticationToken)
-   * @see #setConnectorInfo(Job, String, String)
-   */
-  protected static AuthenticationToken getAuthenticationToken(JobContext job) {
-    AuthenticationToken token =
-        InputConfigurator.getAuthenticationToken(CLASS, job.getConfiguration());
-    return InputConfigurator.unwrapAuthenticationToken(job, token);
-  }
-
-  /**
-   * Configures a {@link org.apache.accumulo.core.client.ZooKeeperInstance} for this job.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param instanceName the Accumulo instance name
-   * @param zooKeepers a comma-separated list of zookeeper servers
-   * @since 1.5.0
-   * @deprecated since 1.6.0
-   */
-  @Deprecated(since = "1.6.0")
-  public static void setZooKeeperInstance(Job job, String instanceName, String zooKeepers) {
-    setZooKeeperInstance(job, org.apache.accumulo.core.client.ClientConfiguration.create()
-        .withInstance(instanceName).withZkHosts(zooKeepers));
-  }
-
-  /**
-   * Configures a {@link org.apache.accumulo.core.client.ZooKeeperInstance} for this job.
-   *
-   * @param job the Hadoop job instance to be configured
-   *
-   * @param clientConfig client configuration containing connection options
-   * @since 1.6.0
-   */
-  public static void setZooKeeperInstance(Job job,
-      org.apache.accumulo.core.client.ClientConfiguration clientConfig) {
-    InputConfigurator.setZooKeeperInstance(CLASS, job.getConfiguration(), clientConfig);
-  }
-
-  /**
-   * Initializes an Accumulo {@link org.apache.accumulo.core.client.Instance} based on the
-   * configuration.
-   *
-   * @param job the Hadoop context for the configured job
-   * @return an Accumulo instance
-   * @since 1.5.0
-   */
-  protected static org.apache.accumulo.core.client.Instance getInstance(JobContext job) {
-    return InputConfigurator.getInstance(CLASS, job.getConfiguration());
-  }
-
-  /**
-   * Sets the log level for this job.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param level the logging level
-   * @since 1.5.0
-   */
-  public static void setLogLevel(Job job, Level level) {
-    InputConfigurator.setLogLevel(CLASS, job.getConfiguration(), level);
-  }
-
-  /**
-   * Gets the log level from this configuration.
-   *
-   * @param job the Hadoop context for the configured job
-   * @return the log level
-   * @since 1.5.0
-   * @see #setLogLevel(Job, Level)
-   */
-  protected static Level getLogLevel(JobContext job) {
-    return InputConfigurator.getLogLevel(CLASS, job.getConfiguration());
-  }
-
-  /**
-   * Sets the {@link Authorizations} used to scan. Must be a subset of the user's authorization.
-   * Defaults to the empty set.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param auths the user's authorizations
-   */
-  public static void setScanAuthorizations(Job job, Authorizations auths) {
-    InputConfigurator.setScanAuthorizations(CLASS, job.getConfiguration(), auths);
-  }
-
-  /**
-   * Gets the authorizations to set for the scans from the configuration.
-   *
-   * @param job the Hadoop context for the configured job
-   * @return the Accumulo scan authorizations
-   * @since 1.5.0
-   * @see #setScanAuthorizations(Job, Authorizations)
-   */
-  protected static Authorizations getScanAuthorizations(JobContext job) {
-    return InputConfigurator.getScanAuthorizations(CLASS, job.getConfiguration());
-  }
-
-  /**
-   * Fetches all {@link InputTableConfig}s that have been set on the given job.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @return the {@link InputTableConfig} objects for the job
-   * @since 1.6.0
-   */
-  protected static Map<String,InputTableConfig> getInputTableConfigs(JobContext job) {
-    return InputConfigurator.getInputTableConfigs(CLASS, job.getConfiguration());
-  }
-
-  /**
-   * Fetches a {@link InputTableConfig} that has been set on the configuration for a specific table.
-   *
-   * <p>
-   * null is returned in the event that the table doesn't exist.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param tableName the table name for which to grab the config object
-   * @return the {@link InputTableConfig} for the given table
-   * @since 1.6.0
-   */
-  protected static InputTableConfig getInputTableConfig(JobContext job, String tableName) {
-    return InputConfigurator.getInputTableConfig(CLASS, job.getConfiguration(), tableName);
-  }
-
-  // InputFormat doesn't have the equivalent of OutputFormat's checkOutputSpecs(JobContext job)
-  /**
-   * Check whether a configuration is fully configured to be used with an Accumulo
-   * {@link org.apache.hadoop.mapreduce.InputFormat}.
-   *
-   * @param job the Hadoop context for the configured job
-   * @throws java.io.IOException if the context is improperly configured
-   * @since 1.5.0
-   */
-  protected static void validateOptions(JobContext job) throws IOException {
-    InputConfigurator.validatePermissions(CLASS, job.getConfiguration());
-  }
-
-  /**
-   * Construct the {@link org.apache.accumulo.core.client.ClientConfiguration} given the provided
-   * context.
-   *
-   * @param job The Job
-   * @return The ClientConfiguration
-   * @since 1.7.0
-   */
-  protected static org.apache.accumulo.core.client.ClientConfiguration
-      getClientConfiguration(JobContext job) {
-    return InputConfigurator.getClientConfiguration(CLASS, job.getConfiguration());
-  }
-
-  /**
-   * An abstract base class to be used to create {@link org.apache.hadoop.mapreduce.RecordReader}
-   * instances that convert from Accumulo {@link Key}/{@link Value} pairs to the user's K/V types.
-   *
-   * Subclasses must implement {@link #nextKeyValue()} and use it to update the following variables:
-   * <ul>
-   * <li>K {@link #currentK}</li>
-   * <li>V {@link #currentV}</li>
-   * <li>Key {@link #currentKey} (used for progress reporting)</li>
-   * <li>int {@link #numKeysRead} (used for progress reporting)</li>
-   * </ul>
-   */
-  protected abstract static class AbstractRecordReader<K,V> extends RecordReader<K,V> {
-    protected long numKeysRead;
-    protected Iterator<Map.Entry<Key,Value>> scannerIterator;
-    protected ScannerBase scannerBase;
-    protected RangeInputSplit split;
-
-    /**
-     * Extracts Iterators settings from the context to be used by RecordReader.
-     *
-     * @param context the Hadoop context for the configured job
-     * @param tableName the table name for which the scanner is configured
-     * @return List of iterator settings for given table
-     * @since 1.7.0
-     */
-    protected abstract List<IteratorSetting> contextIterators(TaskAttemptContext context,
-        String tableName);
-
-    /**
-     * Configures the iterators on a scanner for the given table name. Will attempt to use
-     * configuration from the InputSplit, on failure will try to extract them from
-     * TaskAttemptContext.
-     *
-     * @param context the Hadoop context for the configured job
-     * @param tableName the table name for which the scanner is configured
-     * @param scanner the scanner for which to configure the iterators
-     * @param split InputSplit containing configurations
-     * @since 1.7.0
-     */
-    private void setupIterators(TaskAttemptContext context, ScannerBase scanner, String tableName,
-        RangeInputSplit split) {
-      List<IteratorSetting> iterators = null;
-
-      if (split == null) {
-        iterators = contextIterators(context, tableName);
-      } else {
-        iterators = split.getIterators();
-        if (iterators == null) {
-          iterators = contextIterators(context, tableName);
-        }
-      }
-
-      for (IteratorSetting iterator : iterators) {
-        scanner.addScanIterator(iterator);
-      }
-    }
-
-    /**
-     * Configures the iterators on a scanner for the given table name.
-     *
-     * @param context the Hadoop context for the configured job
-     * @param scanner the scanner for which to configure the iterators
-     * @param tableName the table name for which the scanner is configured
-     * @since 1.6.0
-     * @deprecated since 1.7.0; Use {@link #contextIterators} instead.
-     */
-    @Deprecated(since = "1.7.0")
-    protected void setupIterators(TaskAttemptContext context, Scanner scanner, String tableName,
-        RangeInputSplit split) {
-      setupIterators(context, (ScannerBase) scanner, tableName, split);
-    }
-
-    @Override
-    public void initialize(InputSplit inSplit, TaskAttemptContext attempt) throws IOException {
-
-      split = (RangeInputSplit) inSplit;
-      log.debug("Initializing input split: " + split);
-
-      Authorizations authorizations = getScanAuthorizations(attempt);
-      String classLoaderContext = getClassLoaderContext(attempt);
-      String table = split.getTableName();
-
-      // in case the table name changed, we can still use the previous name for terms of
-      // configuration,
-      // but the scanner will use the table id resolved at job setup time
-      InputTableConfig tableConfig = getInputTableConfig(attempt, split.getTableName());
-
-      ClientContext client = InputConfigurator.client(CLASS, split, attempt.getConfiguration());
-
-      log.debug("Created client with user: " + client.whoami());
-      log.debug("Creating scanner for table: " + table);
-      log.debug("Authorizations are: " + authorizations);
-
-      if (split instanceof org.apache.accumulo.core.clientImpl.mapreduce.BatchInputSplit) {
-        org.apache.accumulo.core.clientImpl.mapreduce.BatchInputSplit batchSplit =
-            (org.apache.accumulo.core.clientImpl.mapreduce.BatchInputSplit) split;
-
-        BatchScanner scanner;
-        try {
-          // Note: BatchScanner will use at most one thread per tablet, currently BatchInputSplit
-          // will not span tablets
-          int scanThreads = 1;
-          scanner = client.createBatchScanner(split.getTableName(), authorizations, scanThreads);
-          setupIterators(attempt, scanner, split.getTableName(), split);
-          if (classLoaderContext != null) {
-            scanner.setClassLoaderContext(classLoaderContext);
-          }
-        } catch (Exception e) {
-          e.printStackTrace();
-          throw new IOException(e);
-        }
-
-        scanner.setRanges(batchSplit.getRanges());
-        scannerBase = scanner;
-      } else {
-        Scanner scanner;
-
-        Boolean isOffline = split.isOffline();
-        if (isOffline == null) {
-          isOffline = tableConfig.isOfflineScan();
-        }
-
-        Boolean isIsolated = split.isIsolatedScan();
-        if (isIsolated == null) {
-          isIsolated = tableConfig.shouldUseIsolatedScanners();
-        }
-
-        Boolean usesLocalIterators = split.usesLocalIterators();
-        if (usesLocalIterators == null) {
-          usesLocalIterators = tableConfig.shouldUseLocalIterators();
-        }
-
-        try {
-          if (isOffline) {
-            scanner = new OfflineScanner(client, TableId.of(split.getTableId()), authorizations);
-          } else {
-            // Not using public API to create scanner so that we can use table ID
-            // Table ID is used in case of renames during M/R job
-            scanner = new ScannerImpl(client, TableId.of(split.getTableId()), authorizations);
-          }
-          if (isIsolated) {
-            log.info("Creating isolated scanner");
-            @SuppressWarnings("resource")
-            var wrapped = new IsolatedScanner(scanner);
-            scanner = wrapped;
-          }
-          if (usesLocalIterators) {
-            log.info("Using local iterators");
-            scanner = new ClientSideIteratorScanner(scanner);
-          }
-
-          setupIterators(attempt, scanner, split.getTableName(), split);
-        } catch (Exception e) {
-          throw new IOException(e);
-        }
-
-        scanner.setRange(split.getRange());
-        scannerBase = scanner;
-
-      }
-
-      Collection<Pair<Text,Text>> columns = split.getFetchedColumns();
-      if (columns == null) {
-        columns = tableConfig.getFetchedColumns();
-      }
-
-      // setup a scanner within the bounds of this split
-      for (Pair<Text,Text> c : columns) {
-        if (c.getSecond() != null) {
-          log.debug("Fetching column " + c.getFirst() + ":" + c.getSecond());
-          scannerBase.fetchColumn(c.getFirst(), c.getSecond());
-        } else {
-          log.debug("Fetching column family " + c.getFirst());
-          scannerBase.fetchColumnFamily(c.getFirst());
-        }
-      }
-
-      SamplerConfiguration samplerConfig = split.getSamplerConfiguration();
-      if (samplerConfig == null) {
-        samplerConfig = tableConfig.getSamplerConfiguration();
-      }
-
-      if (samplerConfig != null) {
-        scannerBase.setSamplerConfiguration(samplerConfig);
-      }
-
-      scannerIterator = scannerBase.iterator();
-      numKeysRead = 0;
-    }
-
-    @Override
-    public void close() {
-      if (scannerBase != null) {
-        scannerBase.close();
-      }
-    }
-
-    @Override
-    public float getProgress() throws IOException {
-      if (numKeysRead > 0 && currentKey == null) {
-        return 1.0f;
-      }
-      return split.getProgress(currentKey);
-    }
-
-    /**
-     * The Key that should be returned to the client
-     */
-    protected K currentK = null;
-
-    /**
-     * The Value that should be return to the client
-     */
-    protected V currentV = null;
-
-    /**
-     * The Key that is used to determine progress in the current InputSplit. It is not returned to
-     * the client and is only used internally
-     */
-    protected Key currentKey = null;
-
-    @Override
-    public K getCurrentKey() throws IOException, InterruptedException {
-      return currentK;
-    }
-
-    @Override
-    public V getCurrentValue() throws IOException, InterruptedException {
-      return currentV;
-    }
-  }
-
-  Map<String,Map<KeyExtent,List<Range>>> binOfflineTable(JobContext job, TableId tableId,
-      List<Range> ranges)
-      throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
-    return InputConfigurator.binOffline(tableId, ranges,
-        InputConfigurator.client(CLASS, job.getConfiguration()));
-  }
-
-  /**
-   * Gets the splits of the tables that have been set on the job by reading the metadata table for
-   * the specified ranges.
-   *
-   * @return the splits from the tables based on the ranges.
-   * @throws java.io.IOException if a table set on the job doesn't exist or an error occurs
-   *         initializing the tablet locator
-   */
-  @Override
-  public List<InputSplit> getSplits(JobContext job) throws IOException {
-    Level logLevel = getLogLevel(job);
-    log.setLevel(logLevel);
-    validateOptions(job);
-
-    LinkedList<InputSplit> splits = new LinkedList<>();
-    Map<String,InputTableConfig> tableConfigs = getInputTableConfigs(job);
-
-    for (Map.Entry<String,InputTableConfig> tableConfigEntry : tableConfigs.entrySet()) {
-
-      String tableName = tableConfigEntry.getKey();
-      InputTableConfig tableConfig = tableConfigEntry.getValue();
-
-      ClientContext client;
-      try {
-        client = InputConfigurator.client(CLASS, job.getConfiguration());
-      } catch (AccumuloException | AccumuloSecurityException e) {
-        throw new IOException(e);
-      }
-
-      TableId tableId;
-      // resolve table name to id once, and use id from this point forward
-      try {
-        tableId = client.getTableId(tableName);
-      } catch (TableNotFoundException e) {
-        throw new IOException(e);
-      }
-
-      boolean batchScan = InputConfigurator.isBatchScan(CLASS, job.getConfiguration());
-      boolean supportBatchScan = !(tableConfig.isOfflineScan()
-          || tableConfig.shouldUseIsolatedScanners() || tableConfig.shouldUseLocalIterators());
-      if (batchScan && !supportBatchScan) {
-        throw new IllegalArgumentException("BatchScanner optimization not available for offline"
-            + " scan, isolated, or local iterators");
-      }
-
-      boolean autoAdjust = tableConfig.shouldAutoAdjustRanges();
-      if (batchScan && !autoAdjust) {
-        throw new IllegalArgumentException(
-            "AutoAdjustRanges must be enabled when using BatchScanner optimization");
-      }
-
-      List<Range> ranges =
-          autoAdjust ? Range.mergeOverlapping(tableConfig.getRanges()) : tableConfig.getRanges();
-      if (ranges.isEmpty()) {
-        ranges = new ArrayList<>(1);
-        ranges.add(new Range());
-      }
-
-      // get the metadata information for these ranges
-      Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<>();
-      TabletLocator tl;
-      try {
-        if (tableConfig.isOfflineScan()) {
-          binnedRanges = binOfflineTable(job, tableId, ranges);
-          while (binnedRanges == null) {
-            // Some tablets were still online, try again
-            // sleep randomly between 100 and 200 ms
-            sleepUninterruptibly(100 + random.nextInt(100), TimeUnit.MILLISECONDS);
-            binnedRanges = binOfflineTable(job, tableId, ranges);
-          }
-        } else {
-          tl = TabletLocator.getLocator(client, tableId);
-          // its possible that the cache could contain complete, but old information about a
-          // tables tablets... so clear it
-          tl.invalidateCache();
-
-          while (!tl.binRanges(client, ranges, binnedRanges).isEmpty()) {
-            client.requireNotDeleted(tableId);
-            client.requireNotOffline(tableId, tableName);
-            binnedRanges.clear();
-            log.warn("Unable to locate bins for specified ranges. Retrying.");
-            // sleep randomly between 100 and 200 ms
-            sleepUninterruptibly(100 + random.nextInt(100), TimeUnit.MILLISECONDS);
-            tl.invalidateCache();
-          }
-        }
-      } catch (Exception e) {
-        throw new IOException(e);
-      }
-
-      // all of this code will add either range per each locations or split ranges and add
-      // range-location split
-      // Map from Range to Array of Locations, we only use this if we're don't split
-      HashMap<Range,ArrayList<String>> splitsToAdd = null;
-
-      if (!autoAdjust) {
-        splitsToAdd = new HashMap<>();
-      }
-
-      HashMap<String,String> hostNameCache = new HashMap<>();
-      for (Map.Entry<String,Map<KeyExtent,List<Range>>> tserverBin : binnedRanges.entrySet()) {
-        String ip = tserverBin.getKey().split(":", 2)[0];
-        String location = hostNameCache.get(ip);
-        if (location == null) {
-          InetAddress inetAddress = InetAddress.getByName(ip);
-          location = inetAddress.getCanonicalHostName();
-          hostNameCache.put(ip, location);
-        }
-        for (Map.Entry<KeyExtent,List<Range>> extentRanges : tserverBin.getValue().entrySet()) {
-          Range ke = extentRanges.getKey().toDataRange();
-          if (batchScan) {
-            // group ranges by tablet to be read by a BatchScanner
-            ArrayList<Range> clippedRanges = new ArrayList<>();
-            for (Range r : extentRanges.getValue()) {
-              clippedRanges.add(ke.clip(r));
-            }
-            org.apache.accumulo.core.clientImpl.mapreduce.BatchInputSplit split =
-                new org.apache.accumulo.core.clientImpl.mapreduce.BatchInputSplit(tableName,
-                    tableId, clippedRanges, new String[] {location});
-            org.apache.accumulo.core.clientImpl.mapreduce.SplitUtils.updateSplit(split, tableConfig,
-                logLevel);
-
-            splits.add(split);
-          } else {
-            // not grouping by tablet
-            for (Range r : extentRanges.getValue()) {
-              if (autoAdjust) {
-                // divide ranges into smaller ranges, based on the tablets
-                RangeInputSplit split = new RangeInputSplit(tableName, tableId.canonical(),
-                    ke.clip(r), new String[] {location});
-                org.apache.accumulo.core.clientImpl.mapreduce.SplitUtils.updateSplit(split,
-                    tableConfig, logLevel);
-                split.setOffline(tableConfig.isOfflineScan());
-                split.setIsolatedScan(tableConfig.shouldUseIsolatedScanners());
-                split.setUsesLocalIterators(tableConfig.shouldUseLocalIterators());
-                splits.add(split);
-              } else {
-                // don't divide ranges
-                ArrayList<String> locations = splitsToAdd.get(r);
-                if (locations == null) {
-                  locations = new ArrayList<>(1);
-                }
-                locations.add(location);
-                splitsToAdd.put(r, locations);
-              }
-            }
-          }
-        }
-      }
-
-      if (!autoAdjust) {
-        for (Map.Entry<Range,ArrayList<String>> entry : splitsToAdd.entrySet()) {
-          RangeInputSplit split = new RangeInputSplit(tableName, tableId.canonical(),
-              entry.getKey(), entry.getValue().toArray(new String[0]));
-          org.apache.accumulo.core.clientImpl.mapreduce.SplitUtils.updateSplit(split, tableConfig,
-              logLevel);
-          split.setOffline(tableConfig.isOfflineScan());
-          split.setIsolatedScan(tableConfig.shouldUseIsolatedScanners());
-          split.setUsesLocalIterators(tableConfig.shouldUseLocalIterators());
-
-          splits.add(split);
-        }
-      }
-    }
-    return splits;
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormat.java
deleted file mode 100644
index a50beab..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormat.java
+++ /dev/null
@@ -1,172 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.client.mapreduce;
-
-import java.io.IOException;
-
-import org.apache.accumulo.core.client.rfile.RFile;
-import org.apache.accumulo.core.client.rfile.RFileWriter;
-import org.apache.accumulo.core.client.sample.SamplerConfiguration;
-import org.apache.accumulo.core.clientImpl.mapreduce.lib.FileOutputConfigurator;
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Value;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.RecordWriter;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.log4j.Logger;
-
-/**
- * This class allows MapReduce jobs to write output in the Accumulo data file format.<br>
- * Care should be taken to write only sorted data (sorted by {@link Key}), as this is an important
- * requirement of Accumulo data files.
- *
- * <p>
- * The output path to be created must be specified via
- * {@link AccumuloFileOutputFormat#setOutputPath(Job, Path)}. This is inherited from
- * {@link FileOutputFormat#setOutputPath(Job, Path)}. Other methods from {@link FileOutputFormat}
- * are not supported and may be ignored or cause failures. Using other Hadoop configuration options
- * that affect the behavior of the underlying files directly in the Job's configuration may work,
- * but are not directly supported at this time.
- *
- * @deprecated since 2.0.0; Use org.apache.accumulo.hadoop.mapreduce instead from the
- *             accumulo-hadoop-mapreduce.jar
- */
-@Deprecated(since = "2.0.0")
-public class AccumuloFileOutputFormat extends FileOutputFormat<Key,Value> {
-
-  private static final Class<?> CLASS = AccumuloFileOutputFormat.class;
-  protected static final Logger log = Logger.getLogger(CLASS);
-
-  /**
-   * Sets the compression type to use for data blocks. Specifying a compression may require
-   * additional libraries to be available to your Job.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param compressionType one of "none", "gz", "bzip2", "lzo", "lz4", "snappy", or "zstd"
-   * @since 1.5.0
-   */
-  public static void setCompressionType(Job job, String compressionType) {
-    FileOutputConfigurator.setCompressionType(CLASS, job.getConfiguration(), compressionType);
-  }
-
-  /**
-   * Sets the size for data blocks within each file.<br>
-   * Data blocks are a span of key/value pairs stored in the file that are compressed and indexed as
-   * a group.
-   *
-   * <p>
-   * Making this value smaller may increase seek performance, but at the cost of increasing the size
-   * of the indexes (which can also affect seek performance).
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param dataBlockSize the block size, in bytes
-   * @since 1.5.0
-   */
-  public static void setDataBlockSize(Job job, long dataBlockSize) {
-    FileOutputConfigurator.setDataBlockSize(CLASS, job.getConfiguration(), dataBlockSize);
-  }
-
-  /**
-   * Sets the size for file blocks in the file system; file blocks are managed, and replicated, by
-   * the underlying file system.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param fileBlockSize the block size, in bytes
-   * @since 1.5.0
-   */
-  public static void setFileBlockSize(Job job, long fileBlockSize) {
-    FileOutputConfigurator.setFileBlockSize(CLASS, job.getConfiguration(), fileBlockSize);
-  }
-
-  /**
-   * Sets the size for index blocks within each file; smaller blocks means a deeper index hierarchy
-   * within the file, while larger blocks mean a more shallow index hierarchy within the file. This
-   * can affect the performance of queries.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param indexBlockSize the block size, in bytes
-   * @since 1.5.0
-   */
-  public static void setIndexBlockSize(Job job, long indexBlockSize) {
-    FileOutputConfigurator.setIndexBlockSize(CLASS, job.getConfiguration(), indexBlockSize);
-  }
-
-  /**
-   * Sets the file system replication factor for the resulting file, overriding the file system
-   * default.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param replication the number of replicas for produced files
-   * @since 1.5.0
-   */
-  public static void setReplication(Job job, int replication) {
-    FileOutputConfigurator.setReplication(CLASS, job.getConfiguration(), replication);
-  }
-
-  /**
-   * Specify a sampler to be used when writing out data. This will result in the output file having
-   * sample data.
-   *
-   * @param job The Hadoop job instance to be configured
-   * @param samplerConfig The configuration for creating sample data in the output file.
-   * @since 1.8.0
-   */
-
-  public static void setSampler(Job job, SamplerConfiguration samplerConfig) {
-    FileOutputConfigurator.setSampler(CLASS, job.getConfiguration(), samplerConfig);
-  }
-
-  @Override
-  public RecordWriter<Key,Value> getRecordWriter(TaskAttemptContext context) throws IOException {
-    // get the path of the temporary output file
-    final Configuration conf = context.getConfiguration();
-    final AccumuloConfiguration acuConf =
-        FileOutputConfigurator.getAccumuloConfiguration(CLASS, context.getConfiguration());
-
-    final String extension = acuConf.get(Property.TABLE_FILE_TYPE);
-    final Path file = this.getDefaultWorkFile(context, "." + extension);
-    final int visCacheSize = FileOutputConfigurator.getVisibilityCacheSize(conf);
-
-    return new RecordWriter<>() {
-      RFileWriter out = null;
-
-      @Override
-      public void close(TaskAttemptContext context) throws IOException {
-        if (out != null) {
-          out.close();
-        }
-      }
-
-      @Override
-      public void write(Key key, Value value) throws IOException {
-        if (out == null) {
-          out = RFile.newWriter().to(file.toString()).withFileSystem(file.getFileSystem(conf))
-              .withTableProperties(acuConf).withVisibilityCacheSize(visCacheSize).build();
-          out.startDefaultLocalityGroup();
-        }
-        out.append(key, value);
-      }
-    };
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormat.java
deleted file mode 100644
index 127a94e..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormat.java
+++ /dev/null
@@ -1,92 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.client.mapreduce;
-
-import java.io.IOException;
-import java.util.Map.Entry;
-
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.format.DefaultFormatter;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.log4j.Level;
-
-/**
- * This class allows MapReduce jobs to use Accumulo as the source of data. This {@link InputFormat}
- * provides keys and values of type {@link Key} and {@link Value} to the Map function.
- *
- * The user must specify the following via static configurator methods:
- *
- * <ul>
- * <li>{@link AccumuloInputFormat#setConnectorInfo(Job, String, AuthenticationToken)}
- * <li>{@link AccumuloInputFormat#setConnectorInfo(Job, String, String)}
- * <li>{@link AccumuloInputFormat#setInputTableName(Job, String)}
- * <li>{@link AccumuloInputFormat#setScanAuthorizations(Job, Authorizations)}
- * </ul>
- *
- * Other static methods are optional.
- *
- * @deprecated since 2.0.0; Use org.apache.accumulo.hadoop.mapreduce instead from the
- *             accumulo-hadoop-mapreduce.jar
- */
-@Deprecated(since = "2.0.0")
-public class AccumuloInputFormat extends InputFormatBase<Key,Value> {
-
-  @Override
-  public RecordReader<Key,Value> createRecordReader(InputSplit split, TaskAttemptContext context)
-      throws IOException, InterruptedException {
-    log.setLevel(getLogLevel(context));
-
-    // Override the log level from the configuration as if the InputSplit has one it's the more
-    // correct one to use.
-    if (split instanceof org.apache.accumulo.core.client.mapreduce.RangeInputSplit) {
-      org.apache.accumulo.core.client.mapreduce.RangeInputSplit accSplit =
-          (org.apache.accumulo.core.client.mapreduce.RangeInputSplit) split;
-      Level level = accSplit.getLogLevel();
-      if (level != null) {
-        log.setLevel(level);
-      }
-    } else {
-      throw new IllegalArgumentException("No RecordReader for " + split.getClass());
-    }
-
-    return new RecordReaderBase<>() {
-      @Override
-      public boolean nextKeyValue() throws IOException, InterruptedException {
-        if (scannerIterator.hasNext()) {
-          ++numKeysRead;
-          Entry<Key,Value> entry = scannerIterator.next();
-          currentK = currentKey = entry.getKey();
-          currentV = entry.getValue();
-          if (log.isTraceEnabled()) {
-            log.trace("Processing key/value pair: " + DefaultFormatter.formatEntry(entry, true));
-          }
-          return true;
-        }
-        return false;
-      }
-    };
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormat.java
deleted file mode 100644
index b3020e5..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormat.java
+++ /dev/null
@@ -1,100 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.client.mapreduce;
-
-import static java.util.Objects.requireNonNull;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.format.DefaultFormatter;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-
-/**
- * This class allows MapReduce jobs to use multiple Accumulo tables as the source of data. This
- * {@link org.apache.hadoop.mapreduce.InputFormat} provides keys and values of type {@link Key} and
- * {@link Value} to the Map function.
- *
- * The user must specify the following via static configurator methods:
- *
- * <ul>
- * <li>{@link AccumuloMultiTableInputFormat#setConnectorInfo(Job, String, AuthenticationToken)}
- * <li>{@link AccumuloMultiTableInputFormat#setConnectorInfo(Job, String, String)}
- * <li>{@link AccumuloMultiTableInputFormat#setScanAuthorizations(Job, Authorizations)}
- * <li>{@link AccumuloMultiTableInputFormat#setInputTableConfigs(Job, Map)}
- * </ul>
- *
- * Other static methods are optional.
- *
- * @deprecated since 2.0.0; Use org.apache.accumulo.hadoop.mapreduce instead from the
- *             accumulo-hadoop-mapreduce.jar
- */
-@Deprecated(since = "2.0.0")
-public class AccumuloMultiTableInputFormat extends AbstractInputFormat<Key,Value> {
-
-  /**
-   * Sets the {@link InputTableConfig} objects on the given Hadoop configuration
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param configs the table query configs to be set on the configuration.
-   * @since 1.6.0
-   */
-  public static void setInputTableConfigs(Job job, Map<String,InputTableConfig> configs) {
-    requireNonNull(configs);
-    org.apache.accumulo.core.clientImpl.mapreduce.lib.InputConfigurator.setInputTableConfigs(CLASS,
-        job.getConfiguration(), configs);
-  }
-
-  @Override
-  public RecordReader<Key,Value> createRecordReader(InputSplit inputSplit,
-      TaskAttemptContext context) throws IOException, InterruptedException {
-    log.setLevel(getLogLevel(context));
-    return new AbstractRecordReader<>() {
-      @Override
-      public boolean nextKeyValue() throws IOException, InterruptedException {
-        if (scannerIterator.hasNext()) {
-          ++numKeysRead;
-          Map.Entry<Key,Value> entry = scannerIterator.next();
-          currentK = currentKey = entry.getKey();
-          currentV = entry.getValue();
-          if (log.isTraceEnabled()) {
-            log.trace("Processing key/value pair: " + DefaultFormatter.formatEntry(entry, true));
-          }
-          return true;
-        }
-        return false;
-      }
-
-      @Override
-      protected List<IteratorSetting> contextIterators(TaskAttemptContext context,
-          String tableName) {
-        return getInputTableConfig(context, tableName).getIterators();
-      }
-    };
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java
deleted file mode 100644
index c39c101..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java
+++ /dev/null
@@ -1,581 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.client.mapreduce;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Set;
-
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.BatchWriterConfig;
-import org.apache.accumulo.core.client.MultiTableBatchWriter;
-import org.apache.accumulo.core.client.MutationsRejectedException;
-import org.apache.accumulo.core.client.TableExistsException;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.admin.DelegationTokenConfig;
-import org.apache.accumulo.core.client.admin.SecurityOperations;
-import org.apache.accumulo.core.client.security.SecurityErrorCode;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken.AuthenticationTokenSerializer;
-import org.apache.accumulo.core.client.security.tokens.DelegationToken;
-import org.apache.accumulo.core.client.security.tokens.KerberosToken;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.clientImpl.AuthenticationTokenIdentifier;
-import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.clientImpl.DelegationTokenImpl;
-import org.apache.accumulo.core.clientImpl.mapreduce.lib.OutputConfigurator;
-import org.apache.accumulo.core.data.ColumnUpdate;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.security.ColumnVisibility;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.OutputCommitter;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.hadoop.mapreduce.RecordWriter;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
-import org.apache.hadoop.security.token.Token;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
-
-/**
- * This class allows MapReduce jobs to use Accumulo as the sink for data. This {@link OutputFormat}
- * accepts keys and values of type {@link Text} (for a table name) and {@link Mutation} from the Map
- * and Reduce functions.
- *
- * The user must specify the following via static configurator methods:
- *
- * <ul>
- * <li>{@link AccumuloOutputFormat#setConnectorInfo(Job, String, AuthenticationToken)}
- * <li>{@link AccumuloOutputFormat#setConnectorInfo(Job, String, String)}
- * <li>{@link AccumuloOutputFormat#setZooKeeperInstance(Job, org.apache.accumulo.core.client.ClientConfiguration)}
- * </ul>
- *
- * Other static methods are optional.
- *
- * @deprecated since 2.0.0; Use org.apache.accumulo.hadoop.mapreduce instead from the
- *             accumulo-hadoop-mapreduce.jar
- */
-@Deprecated(since = "2.0.0")
-public class AccumuloOutputFormat extends OutputFormat<Text,Mutation> {
-
-  private static final Class<?> CLASS = AccumuloOutputFormat.class;
-  protected static final Logger log = Logger.getLogger(CLASS);
-
-  /**
-   * Sets the connector information needed to communicate with Accumulo in this job.
-   *
-   * <p>
-   * <b>WARNING:</b> Some tokens, when serialized, divulge sensitive information in the
-   * configuration as a means to pass the token to MapReduce tasks. This information is BASE64
-   * encoded to provide a charset safe conversion to a string, but this conversion is not intended
-   * to be secure. {@link PasswordToken} is one example that is insecure in this way; however
-   * {@link DelegationToken}s, acquired using
-   * {@link SecurityOperations#getDelegationToken(DelegationTokenConfig)}, is not subject to this
-   * concern.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param principal a valid Accumulo user name (user must have Table.CREATE permission if
-   *        {@link #setCreateTables(Job, boolean)} is set to true)
-   * @param token the user's password
-   * @since 1.5.0
-   */
-  public static void setConnectorInfo(Job job, String principal, AuthenticationToken token)
-      throws AccumuloSecurityException {
-    if (token instanceof KerberosToken) {
-      log.info("Received KerberosToken, attempting to fetch DelegationToken");
-      try {
-        ClientContext client = OutputConfigurator.client(CLASS, job.getConfiguration());
-        token = client.securityOperations().getDelegationToken(new DelegationTokenConfig());
-      } catch (Exception e) {
-        log.warn("Failed to automatically obtain DelegationToken, "
-            + "Mappers/Reducers will likely fail to communicate with Accumulo", e);
-      }
-    }
-    // DelegationTokens can be passed securely from user to task without serializing insecurely in
-    // the configuration
-    if (token instanceof DelegationTokenImpl) {
-      DelegationTokenImpl delegationToken = (DelegationTokenImpl) token;
-
-      // Convert it into a Hadoop Token
-      AuthenticationTokenIdentifier identifier = delegationToken.getIdentifier();
-      Token<AuthenticationTokenIdentifier> hadoopToken = new Token<>(identifier.getBytes(),
-          delegationToken.getPassword(), identifier.getKind(), delegationToken.getServiceName());
-
-      // Add the Hadoop Token to the Job so it gets serialized and passed along.
-      job.getCredentials().addToken(hadoopToken.getService(), hadoopToken);
-    }
-
-    OutputConfigurator.setConnectorInfo(CLASS, job.getConfiguration(), principal, token);
-  }
-
-  /**
-   * Sets the connector information needed to communicate with Accumulo in this job.
-   *
-   * <p>
-   * Stores the password in a file in HDFS and pulls that into the Distributed Cache in an attempt
-   * to be more secure than storing it in the Configuration.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param principal a valid Accumulo user name (user must have Table.CREATE permission if
-   *        {@link #setCreateTables(Job, boolean)} is set to true)
-   * @param tokenFile the path to the token file
-   * @since 1.6.0
-   */
-  public static void setConnectorInfo(Job job, String principal, String tokenFile)
-      throws AccumuloSecurityException {
-    OutputConfigurator.setConnectorInfo(CLASS, job.getConfiguration(), principal, tokenFile);
-  }
-
-  /**
-   * Determines if the connector has been configured.
-   *
-   * @param context the Hadoop context for the configured job
-   * @return true if the connector has been configured, false otherwise
-   * @since 1.5.0
-   * @see #setConnectorInfo(Job, String, AuthenticationToken)
-   */
-  protected static Boolean isConnectorInfoSet(JobContext context) {
-    return OutputConfigurator.isConnectorInfoSet(CLASS, context.getConfiguration());
-  }
-
-  /**
-   * Gets the user name from the configuration.
-   *
-   * @param context the Hadoop context for the configured job
-   * @return the user name
-   * @since 1.5.0
-   * @see #setConnectorInfo(Job, String, AuthenticationToken)
-   */
-  protected static String getPrincipal(JobContext context) {
-    return OutputConfigurator.getPrincipal(CLASS, context.getConfiguration());
-  }
-
-  /**
-   * Gets the serialized token class from either the configuration or the token file.
-   *
-   * @since 1.5.0
-   * @deprecated since 1.6.0; Use {@link #getAuthenticationToken(JobContext)} instead.
-   */
-  @Deprecated(since = "1.6.0")
-  protected static String getTokenClass(JobContext context) {
-    return getAuthenticationToken(context).getClass().getName();
-  }
-
-  /**
-   * Gets the serialized token from either the configuration or the token file.
-   *
-   * @since 1.5.0
-   * @deprecated since 1.6.0; Use {@link #getAuthenticationToken(JobContext)} instead.
-   */
-  @Deprecated(since = "1.6.0")
-  protected static byte[] getToken(JobContext context) {
-    return AuthenticationTokenSerializer.serialize(getAuthenticationToken(context));
-  }
-
-  /**
-   * Gets the authenticated token from either the specified token file or directly from the
-   * configuration, whichever was used when the job was configured.
-   *
-   * @param context the Hadoop context for the configured job
-   * @return the principal's authentication token
-   * @since 1.6.0
-   * @see #setConnectorInfo(Job, String, AuthenticationToken)
-   * @see #setConnectorInfo(Job, String, String)
-   */
-  protected static AuthenticationToken getAuthenticationToken(JobContext context) {
-    AuthenticationToken token =
-        OutputConfigurator.getAuthenticationToken(CLASS, context.getConfiguration());
-    return OutputConfigurator.unwrapAuthenticationToken(context, token);
-  }
-
-  /**
-   * Configures a {@link org.apache.accumulo.core.client.ZooKeeperInstance} for this job.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param instanceName the Accumulo instance name
-   * @param zooKeepers a comma-separated list of zookeeper servers
-   * @since 1.5.0
-   * @deprecated since 1.6.0; Use
-   *             {@link #setZooKeeperInstance(Job, org.apache.accumulo.core.client.ClientConfiguration)}
-   *             instead.
-   */
-  @Deprecated(since = "1.6.0")
-  public static void setZooKeeperInstance(Job job, String instanceName, String zooKeepers) {
-    setZooKeeperInstance(job, org.apache.accumulo.core.client.ClientConfiguration.create()
-        .withInstance(instanceName).withZkHosts(zooKeepers));
-  }
-
-  /**
-   * Configures a {@link org.apache.accumulo.core.client.ZooKeeperInstance} for this job.
-   *
-   * @param job the Hadoop job instance to be configured
-   *
-   * @param clientConfig client configuration for specifying connection timeouts, SSL connection
-   *        options, etc.
-   * @since 1.6.0
-   */
-  public static void setZooKeeperInstance(Job job,
-      org.apache.accumulo.core.client.ClientConfiguration clientConfig) {
-    OutputConfigurator.setZooKeeperInstance(CLASS, job.getConfiguration(), clientConfig);
-  }
-
-  /**
-   * Initializes an Accumulo {@link org.apache.accumulo.core.client.Instance} based on the
-   * configuration.
-   *
-   * @param context the Hadoop context for the configured job
-   * @return an Accumulo instance
-   * @since 1.5.0
-   */
-  protected static org.apache.accumulo.core.client.Instance getInstance(JobContext context) {
-    return OutputConfigurator.getInstance(CLASS, context.getConfiguration());
-  }
-
-  /**
-   * Sets the log level for this job.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param level the logging level
-   * @since 1.5.0
-   */
-  public static void setLogLevel(Job job, Level level) {
-    OutputConfigurator.setLogLevel(CLASS, job.getConfiguration(), level);
-  }
-
-  /**
-   * Gets the log level from this configuration.
-   *
-   * @param context the Hadoop context for the configured job
-   * @return the log level
-   * @since 1.5.0
-   * @see #setLogLevel(Job, Level)
-   */
-  protected static Level getLogLevel(JobContext context) {
-    return OutputConfigurator.getLogLevel(CLASS, context.getConfiguration());
-  }
-
-  /**
-   * Sets the default table name to use if one emits a null in place of a table name for a given
-   * mutation. Table names can only be alpha-numeric and underscores.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param tableName the table to use when the tablename is null in the write call
-   * @since 1.5.0
-   */
-  public static void setDefaultTableName(Job job, String tableName) {
-    OutputConfigurator.setDefaultTableName(CLASS, job.getConfiguration(), tableName);
-  }
-
-  /**
-   * Gets the default table name from the configuration.
-   *
-   * @param context the Hadoop context for the configured job
-   * @return the default table name
-   * @since 1.5.0
-   * @see #setDefaultTableName(Job, String)
-   */
-  protected static String getDefaultTableName(JobContext context) {
-    return OutputConfigurator.getDefaultTableName(CLASS, context.getConfiguration());
-  }
-
-  /**
-   * Sets the configuration for for the job's {@link BatchWriter} instances. If not set, a new
-   * {@link BatchWriterConfig}, with sensible built-in defaults is used. Setting the configuration
-   * multiple times overwrites any previous configuration.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param bwConfig the configuration for the {@link BatchWriter}
-   * @since 1.5.0
-   */
-  public static void setBatchWriterOptions(Job job, BatchWriterConfig bwConfig) {
-    OutputConfigurator.setBatchWriterOptions(CLASS, job.getConfiguration(), bwConfig);
-  }
-
-  /**
-   * Gets the {@link BatchWriterConfig} settings.
-   *
-   * @param context the Hadoop context for the configured job
-   * @return the configuration object
-   * @since 1.5.0
-   * @see #setBatchWriterOptions(Job, BatchWriterConfig)
-   */
-  protected static BatchWriterConfig getBatchWriterOptions(JobContext context) {
-    return OutputConfigurator.getBatchWriterOptions(CLASS, context.getConfiguration());
-  }
-
-  /**
-   * Sets the directive to create new tables, as necessary. Table names can only be alpha-numeric
-   * and underscores.
-   *
-   * <p>
-   * By default, this feature is <b>disabled</b>.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param enableFeature the feature is enabled if true, disabled otherwise
-   * @since 1.5.0
-   */
-  public static void setCreateTables(Job job, boolean enableFeature) {
-    OutputConfigurator.setCreateTables(CLASS, job.getConfiguration(), enableFeature);
-  }
-
-  /**
-   * Determines whether tables are permitted to be created as needed.
-   *
-   * @param context the Hadoop context for the configured job
-   * @return true if the feature is disabled, false otherwise
-   * @since 1.5.0
-   * @see #setCreateTables(Job, boolean)
-   */
-  protected static Boolean canCreateTables(JobContext context) {
-    return OutputConfigurator.canCreateTables(CLASS, context.getConfiguration());
-  }
-
-  /**
-   * Sets the directive to use simulation mode for this job. In simulation mode, no output is
-   * produced. This is useful for testing.
-   *
-   * <p>
-   * By default, this feature is <b>disabled</b>.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param enableFeature the feature is enabled if true, disabled otherwise
-   * @since 1.5.0
-   */
-  public static void setSimulationMode(Job job, boolean enableFeature) {
-    OutputConfigurator.setSimulationMode(CLASS, job.getConfiguration(), enableFeature);
-  }
-
-  /**
-   * Determines whether this feature is enabled.
-   *
-   * @param context the Hadoop context for the configured job
-   * @return true if the feature is enabled, false otherwise
-   * @since 1.5.0
-   * @see #setSimulationMode(Job, boolean)
-   */
-  protected static Boolean getSimulationMode(JobContext context) {
-    return OutputConfigurator.getSimulationMode(CLASS, context.getConfiguration());
-  }
-
-  /**
-   * A base class to be used to create {@link RecordWriter} instances that write to Accumulo.
-   */
-  protected static class AccumuloRecordWriter extends RecordWriter<Text,Mutation> {
-    private MultiTableBatchWriter mtbw = null;
-    private HashMap<Text,BatchWriter> bws = null;
-    private Text defaultTableName = null;
-
-    private boolean simulate = false;
-    private boolean createTables = false;
-
-    private long mutCount = 0;
-    private long valCount = 0;
-
-    private AccumuloClient client;
-
-    protected AccumuloRecordWriter(TaskAttemptContext context)
-        throws AccumuloException, AccumuloSecurityException, IOException {
-      Level l = getLogLevel(context);
-      if (l != null) {
-        log.setLevel(getLogLevel(context));
-      }
-      this.simulate = getSimulationMode(context);
-      this.createTables = canCreateTables(context);
-
-      if (simulate) {
-        log.info("Simulating output only. No writes to tables will occur");
-      }
-
-      this.bws = new HashMap<>();
-
-      String tname = getDefaultTableName(context);
-      this.defaultTableName = (tname == null) ? null : new Text(tname);
-
-      if (!simulate) {
-        this.client = OutputConfigurator.client(CLASS, context.getConfiguration());
-        mtbw = client.createMultiTableBatchWriter(getBatchWriterOptions(context));
-      }
-    }
-
-    /**
-     * Push a mutation into a table. If table is null, the defaultTable will be used. If
-     * {@link AccumuloOutputFormat#canCreateTables(JobContext)} is set, the table will be created if
-     * it does not exist. The table name must only contain alphanumerics and underscore.
-     */
-    @Override
-    public void write(Text table, Mutation mutation) throws IOException {
-      if (table == null || table.getLength() == 0) {
-        table = this.defaultTableName;
-      }
-
-      if (!simulate && table == null) {
-        throw new IOException("No table or default table specified. Try simulation mode next time");
-      }
-
-      ++mutCount;
-      valCount += mutation.size();
-      printMutation(table, mutation);
-
-      if (simulate) {
-        return;
-      }
-
-      if (!bws.containsKey(table)) {
-        try {
-          addTable(table);
-        } catch (Exception e) {
-          log.error("Could not add table '" + table + "'", e);
-          throw new IOException(e);
-        }
-      }
-
-      try {
-        bws.get(table).addMutation(mutation);
-      } catch (MutationsRejectedException e) {
-        throw new IOException(e);
-      }
-    }
-
-    public void addTable(Text tableName) throws AccumuloException, AccumuloSecurityException {
-      if (simulate) {
-        log.info("Simulating adding table: " + tableName);
-        return;
-      }
-
-      log.debug("Adding table: " + tableName);
-      BatchWriter bw = null;
-      String table = tableName.toString();
-
-      if (createTables && !client.tableOperations().exists(table)) {
-        try {
-          client.tableOperations().create(table);
-        } catch (AccumuloSecurityException e) {
-          log.error("Accumulo security violation creating " + table, e);
-          throw e;
-        } catch (TableExistsException e) {
-          // Shouldn't happen
-        }
-      }
-
-      try {
-        bw = mtbw.getBatchWriter(table);
-      } catch (TableNotFoundException e) {
-        log.error("Accumulo table " + table + " doesn't exist and cannot be created.", e);
-        throw new AccumuloException(e);
-      } catch (AccumuloException | AccumuloSecurityException e) {
-        throw e;
-      }
-
-      if (bw != null) {
-        bws.put(tableName, bw);
-      }
-    }
-
-    private int printMutation(Text table, Mutation m) {
-      if (log.isTraceEnabled()) {
-        log.trace(String.format("Table %s row key: %s", table, hexDump(m.getRow())));
-        for (ColumnUpdate cu : m.getUpdates()) {
-          log.trace(String.format("Table %s column: %s:%s", table, hexDump(cu.getColumnFamily()),
-              hexDump(cu.getColumnQualifier())));
-          log.trace(String.format("Table %s security: %s", table,
-              new ColumnVisibility(cu.getColumnVisibility())));
-          log.trace(String.format("Table %s value: %s", table, hexDump(cu.getValue())));
-        }
-      }
-      return m.getUpdates().size();
-    }
-
-    private String hexDump(byte[] ba) {
-      StringBuilder sb = new StringBuilder();
-      for (byte b : ba) {
-        if ((b > 0x20) && (b < 0x7e)) {
-          sb.append((char) b);
-        } else {
-          sb.append(String.format("x%02x", b));
-        }
-      }
-      return sb.toString();
-    }
-
-    @Override
-    public void close(TaskAttemptContext attempt) throws IOException, InterruptedException {
-      log.debug("mutations written: " + mutCount + ", values written: " + valCount);
-      if (simulate) {
-        return;
-      }
-
-      try {
-        mtbw.close();
-      } catch (MutationsRejectedException e) {
-        if (!e.getSecurityErrorCodes().isEmpty()) {
-          var tables = new HashMap<String,Set<SecurityErrorCode>>();
-          e.getSecurityErrorCodes().forEach((table, code) -> tables
-              .computeIfAbsent(table.getTable().canonical(), k -> new HashSet<>()).addAll(code));
-          log.error("Not authorized to write to tables : " + tables);
-        }
-
-        if (!e.getConstraintViolationSummaries().isEmpty()) {
-          log.error("Constraint violations : " + e.getConstraintViolationSummaries().size());
-        }
-        throw new IOException(e);
-      } finally {
-        client.close();
-      }
-    }
-  }
-
-  @Override
-  public void checkOutputSpecs(JobContext job) throws IOException {
-    if (!isConnectorInfoSet(job)) {
-      throw new IOException("Connector info has not been set.");
-    }
-    String principal = getPrincipal(job);
-    AuthenticationToken token = getAuthenticationToken(job);
-    try {
-      AccumuloClient c = OutputConfigurator.client(CLASS, job.getConfiguration());
-      if (!c.securityOperations().authenticateUser(principal, token)) {
-        throw new IOException("Unable to authenticate user");
-      }
-    } catch (AccumuloException | AccumuloSecurityException e) {
-      throw new IOException(e);
-    }
-  }
-
-  @Override
-  public OutputCommitter getOutputCommitter(TaskAttemptContext context) {
-    return new NullOutputFormat<Text,Mutation>().getOutputCommitter(context);
-  }
-
-  @Override
-  public RecordWriter<Text,Mutation> getRecordWriter(TaskAttemptContext attempt)
-      throws IOException {
-    try {
-      return new AccumuloRecordWriter(attempt);
-    } catch (Exception e) {
-      throw new IOException(e);
-    }
-  }
-
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormat.java
deleted file mode 100644
index 0a97744..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormat.java
+++ /dev/null
@@ -1,88 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.client.mapreduce;
-
-import java.io.IOException;
-import java.util.Map.Entry;
-
-import org.apache.accumulo.core.client.RowIterator;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.PeekingIterator;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-
-/**
- * This class allows MapReduce jobs to use Accumulo as the source of data. This {@link InputFormat}
- * provides row names as {@link Text} as keys, and a corresponding {@link PeekingIterator} as a
- * value, which in turn makes the {@link Key}/{@link Value} pairs for that row available to the Map
- * function.
- *
- * The user must specify the following via static configurator methods:
- *
- * <ul>
- * <li>{@link AccumuloRowInputFormat#setConnectorInfo(Job, String, AuthenticationToken)}
- * <li>{@link AccumuloRowInputFormat#setConnectorInfo(Job, String, String)}
- * <li>{@link AccumuloRowInputFormat#setInputTableName(Job, String)}
- * <li>{@link AccumuloRowInputFormat#setScanAuthorizations(Job, Authorizations)}
- * </ul>
- *
- * Other static methods are optional.
- *
- * @deprecated since 2.0.0; Use org.apache.accumulo.hadoop.mapreduce instead from the
- *             accumulo-hadoop-mapreduce.jar
- */
-@Deprecated(since = "2.0.0")
-public class AccumuloRowInputFormat
-    extends InputFormatBase<Text,PeekingIterator<Entry<Key,Value>>> {
-  @Override
-  public RecordReader<Text,PeekingIterator<Entry<Key,Value>>> createRecordReader(InputSplit split,
-      TaskAttemptContext context) throws IOException, InterruptedException {
-    log.setLevel(getLogLevel(context));
-    return new RecordReaderBase<>() {
-      RowIterator rowIterator;
-
-      @Override
-      public void initialize(InputSplit inSplit, TaskAttemptContext attempt) throws IOException {
-        super.initialize(inSplit, attempt);
-        rowIterator = new RowIterator(scannerIterator);
-        currentK = new Text();
-        currentV = null;
-      }
-
-      @Override
-      public boolean nextKeyValue() throws IOException, InterruptedException {
-        if (!rowIterator.hasNext()) {
-          return false;
-        }
-        currentV = new PeekingIterator<>(rowIterator.next());
-        numKeysRead = rowIterator.getKVCount();
-        currentKey = currentV.peek().getKey();
-        currentK = new Text(currentKey.getRow());
-        return true;
-      }
-    };
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
deleted file mode 100644
index f4e4023..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
+++ /dev/null
@@ -1,386 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.client.mapreduce;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.accumulo.core.client.BatchScanner;
-import org.apache.accumulo.core.client.ClientSideIteratorScanner;
-import org.apache.accumulo.core.client.IsolatedScanner;
-import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.ScannerBase;
-import org.apache.accumulo.core.client.admin.TableOperations;
-import org.apache.accumulo.core.client.sample.SamplerConfiguration;
-import org.apache.accumulo.core.clientImpl.mapreduce.lib.InputConfigurator;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.util.Pair;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-
-/**
- * This abstract {@link InputFormat} class allows MapReduce jobs to use Accumulo as the source of
- * K,V pairs.
- * <p>
- * Subclasses must implement a {@link #createRecordReader(InputSplit, TaskAttemptContext)} to
- * provide a {@link RecordReader} for K,V.
- * <p>
- * A static base class, RecordReaderBase, is provided to retrieve Accumulo {@link Key}/{@link Value}
- * pairs, but one must implement its {@link RecordReaderBase#nextKeyValue()} to transform them to
- * the desired generic types K,V.
- * <p>
- * See {@link AccumuloInputFormat} for an example implementation.
- *
- * @deprecated since 2.0.0; Use org.apache.accumulo.hadoop.mapreduce instead from the
- *             accumulo-hadoop-mapreduce.jar
- */
-@Deprecated(since = "2.0.0")
-public abstract class InputFormatBase<K,V> extends AbstractInputFormat<K,V> {
-
-  /**
-   * Gets the table name from the configuration.
-   *
-   * @param context the Hadoop context for the configured job
-   * @return the table name
-   * @since 1.5.0
-   * @see #setInputTableName(Job, String)
-   */
-  protected static String getInputTableName(JobContext context) {
-    return InputConfigurator.getInputTableName(CLASS, context.getConfiguration());
-  }
-
-  /**
-   * Sets the name of the input table, over which this job will scan.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param tableName the table to use when the tablename is null in the write call
-   * @since 1.5.0
-   */
-  public static void setInputTableName(Job job, String tableName) {
-    InputConfigurator.setInputTableName(CLASS, job.getConfiguration(), tableName);
-  }
-
-  /**
-   * Sets the input ranges to scan for the single input table associated with this job.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param ranges the ranges that will be mapped over
-   * @see TableOperations#splitRangeByTablets(String, Range, int)
-   * @since 1.5.0
-   */
-  public static void setRanges(Job job, Collection<Range> ranges) {
-    InputConfigurator.setRanges(CLASS, job.getConfiguration(), ranges);
-  }
-
-  /**
-   * Gets the ranges to scan over from a job.
-   *
-   * @param context the Hadoop context for the configured job
-   * @return the ranges
-   * @since 1.5.0
-   * @see #setRanges(Job, Collection)
-   */
-  protected static List<Range> getRanges(JobContext context) throws IOException {
-    return InputConfigurator.getRanges(CLASS, context.getConfiguration());
-  }
-
-  /**
-   * Restricts the columns that will be mapped over for this job for the default input table.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param columnFamilyColumnQualifierPairs a pair of {@link Text} objects corresponding to column
-   *        family and column qualifier. If the column qualifier is null, the entire column family
-   *        is selected. An empty set is the default and is equivalent to scanning the all columns.
-   * @since 1.5.0
-   */
-  public static void fetchColumns(Job job,
-      Collection<Pair<Text,Text>> columnFamilyColumnQualifierPairs) {
-    InputConfigurator.fetchColumns(CLASS, job.getConfiguration(), columnFamilyColumnQualifierPairs);
-  }
-
-  /**
-   * Gets the columns to be mapped over from this job.
-   *
-   * @param context the Hadoop context for the configured job
-   * @return a set of columns
-   * @since 1.5.0
-   * @see #fetchColumns(Job, Collection)
-   */
-  protected static Set<Pair<Text,Text>> getFetchedColumns(JobContext context) {
-    return InputConfigurator.getFetchedColumns(CLASS, context.getConfiguration());
-  }
-
-  /**
-   * Encode an iterator on the single input table for this job.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param cfg the configuration of the iterator
-   * @since 1.5.0
-   */
-  public static void addIterator(Job job, IteratorSetting cfg) {
-    InputConfigurator.addIterator(CLASS, job.getConfiguration(), cfg);
-  }
-
-  /**
-   * Gets a list of the iterator settings (for iterators to apply to a scanner) from this
-   * configuration.
-   *
-   * @param context the Hadoop context for the configured job
-   * @return a list of iterators
-   * @since 1.5.0
-   * @see #addIterator(Job, IteratorSetting)
-   */
-  protected static List<IteratorSetting> getIterators(JobContext context) {
-    return InputConfigurator.getIterators(CLASS, context.getConfiguration());
-  }
-
-  /**
-   * Controls the automatic adjustment of ranges for this job. This feature merges overlapping
-   * ranges, then splits them to align with tablet boundaries. Disabling this feature will cause
-   * exactly one Map task to be created for each specified range. The default setting is enabled. *
-   *
-   * <p>
-   * By default, this feature is <b>enabled</b>.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param enableFeature the feature is enabled if true, disabled otherwise
-   * @see #setRanges(Job, Collection)
-   * @since 1.5.0
-   */
-  public static void setAutoAdjustRanges(Job job, boolean enableFeature) {
-    InputConfigurator.setAutoAdjustRanges(CLASS, job.getConfiguration(), enableFeature);
-  }
-
-  /**
-   * Determines whether a configuration has auto-adjust ranges enabled. Must be enabled when
-   * {@link #setBatchScan(Job, boolean)} is true.
-   *
-   * @param context the Hadoop context for the configured job
-   * @return false if the feature is disabled, true otherwise
-   * @since 1.5.0
-   * @see #setAutoAdjustRanges(Job, boolean)
-   */
-  protected static boolean getAutoAdjustRanges(JobContext context) {
-    return InputConfigurator.getAutoAdjustRanges(CLASS, context.getConfiguration());
-  }
-
-  /**
-   * Controls the use of the {@link IsolatedScanner} in this job.
-   *
-   * <p>
-   * By default, this feature is <b>disabled</b>.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param enableFeature the feature is enabled if true, disabled otherwise
-   * @since 1.5.0
-   */
-  public static void setScanIsolation(Job job, boolean enableFeature) {
-    InputConfigurator.setScanIsolation(CLASS, job.getConfiguration(), enableFeature);
-  }
-
-  /**
-   * Determines whether a configuration has isolation enabled.
-   *
-   * @param context the Hadoop context for the configured job
-   * @return true if the feature is enabled, false otherwise
-   * @since 1.5.0
-   * @see #setScanIsolation(Job, boolean)
-   */
-  protected static boolean isIsolated(JobContext context) {
-    return InputConfigurator.isIsolated(CLASS, context.getConfiguration());
-  }
-
-  /**
-   * Controls the use of the {@link ClientSideIteratorScanner} in this job. Enabling this feature
-   * will cause the iterator stack to be constructed within the Map task, rather than within the
-   * Accumulo TServer. To use this feature, all classes needed for those iterators must be available
-   * on the classpath for the task.
-   *
-   * <p>
-   * By default, this feature is <b>disabled</b>.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param enableFeature the feature is enabled if true, disabled otherwise
-   * @since 1.5.0
-   */
-  public static void setLocalIterators(Job job, boolean enableFeature) {
-    InputConfigurator.setLocalIterators(CLASS, job.getConfiguration(), enableFeature);
-  }
-
-  /**
-   * Determines whether a configuration uses local iterators.
-   *
-   * @param context the Hadoop context for the configured job
-   * @return true if the feature is enabled, false otherwise
-   * @since 1.5.0
-   * @see #setLocalIterators(Job, boolean)
-   */
-  protected static boolean usesLocalIterators(JobContext context) {
-    return InputConfigurator.usesLocalIterators(CLASS, context.getConfiguration());
-  }
-
-  /**
-   * Enable reading offline tables. By default, this feature is disabled and only online tables are
-   * scanned. This will make the map reduce job directly read the table's files. If the table is not
-   * offline, then the job will fail. If the table comes online during the map reduce job, it is
-   * likely that the job will fail.
-   *
-   * <p>
-   * To use this option, the map reduce user will need access to read the Accumulo directory in
-   * HDFS.
-   *
-   * <p>
-   * Reading the offline table will create the scan time iterator stack in the map process. So any
-   * iterators that are configured for the table will need to be on the mapper's classpath.
-   *
-   * <p>
-   * One way to use this feature is to clone a table, take the clone offline, and use the clone as
-   * the input table for a map reduce job. If you plan to map reduce over the data many times, it
-   * may be better to the compact the table, clone it, take it offline, and use the clone for all
-   * map reduce jobs. The reason to do this is that compaction will reduce each tablet in the table
-   * to one file, and it is faster to read from one file.
-   *
-   * <p>
-   * There are two possible advantages to reading a tables file directly out of HDFS. First, you may
-   * see better read performance. Second, it will support speculative execution better. When reading
-   * an online table speculative execution can put more load on an already slow tablet server.
-   *
-   * <p>
-   * By default, this feature is <b>disabled</b>.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param enableFeature the feature is enabled if true, disabled otherwise
-   * @since 1.5.0
-   */
-  public static void setOfflineTableScan(Job job, boolean enableFeature) {
-    InputConfigurator.setOfflineTableScan(CLASS, job.getConfiguration(), enableFeature);
-  }
-
-  /**
-   * Determines whether a configuration has the offline table scan feature enabled.
-   *
-   * @param context the Hadoop context for the configured job
-   * @return true if the feature is enabled, false otherwise
-   * @since 1.5.0
-   * @see #setOfflineTableScan(Job, boolean)
-   */
-  protected static boolean isOfflineScan(JobContext context) {
-    return InputConfigurator.isOfflineScan(CLASS, context.getConfiguration());
-  }
-
-  /**
-   * Controls the use of the {@link BatchScanner} in this job. Using this feature will group Ranges
-   * by their source tablet, producing an InputSplit per tablet rather than per Range. This batching
-   * helps to reduce overhead when querying a large number of small ranges. (ex: when doing
-   * quad-tree decomposition for spatial queries)
-   * <p>
-   * In order to achieve good locality of InputSplits this option always clips the input Ranges to
-   * tablet boundaries. This may result in one input Range contributing to several InputSplits.
-   * <p>
-   * Note: that the value of {@link #setAutoAdjustRanges(Job, boolean)} is ignored and is assumed to
-   * be true when BatchScan option is enabled.
-   * <p>
-   * This configuration is incompatible with:
-   * <ul>
-   * <li>{@link #setOfflineTableScan(org.apache.hadoop.mapreduce.Job, boolean)}</li>
-   * <li>{@link #setLocalIterators(org.apache.hadoop.mapreduce.Job, boolean)}</li>
-   * <li>{@link #setScanIsolation(org.apache.hadoop.mapreduce.Job, boolean)}</li>
-   * </ul>
-   * <p>
-   * By default, this feature is <b>disabled</b>.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param enableFeature the feature is enabled if true, disabled otherwise
-   * @since 1.7.0
-   */
-  public static void setBatchScan(Job job, boolean enableFeature) {
-    InputConfigurator.setBatchScan(CLASS, job.getConfiguration(), enableFeature);
-  }
-
-  /**
-   * Determines whether a configuration has the {@link BatchScanner} feature enabled.
-   *
-   * @param context the Hadoop context for the configured job
-   * @since 1.7.0
-   * @see #setBatchScan(Job, boolean)
-   */
-  public static boolean isBatchScan(JobContext context) {
-    return InputConfigurator.isBatchScan(CLASS, context.getConfiguration());
-  }
-
-  /**
-   * Causes input format to read sample data. If sample data was created using a different
-   * configuration or a tables sampler configuration changes while reading data, then the input
-   * format will throw an error.
-   *
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param samplerConfig The sampler configuration that sample must have been created with inorder
-   *        for reading sample data to succeed.
-   *
-   * @since 1.8.0
-   * @see ScannerBase#setSamplerConfiguration(SamplerConfiguration)
-   */
-  public static void setSamplerConfiguration(Job job, SamplerConfiguration samplerConfig) {
-    InputConfigurator.setSamplerConfiguration(CLASS, job.getConfiguration(), samplerConfig);
-  }
-
-  protected abstract static class RecordReaderBase<K,V> extends AbstractRecordReader<K,V> {
-
-    @Override
-    @Deprecated(since = "2.0.0")
-    protected List<IteratorSetting> contextIterators(TaskAttemptContext context, String tableName) {
-      return getIterators(context);
-    }
-
-    /**
-     * Apply the configured iterators from the configuration to the scanner.
-     *
-     * @param context the Hadoop context for the configured job
-     * @param scanner the scanner to configure
-     * @deprecated since 1.7.0; Use {@link #contextIterators} instead.
-     */
-    @Deprecated(since = "1.7.0")
-    protected void setupIterators(TaskAttemptContext context, Scanner scanner) {
-      // tableName is given as null as it will be ignored in eventual call to #contextIterators
-      setupIterators(context, scanner, null, null);
-    }
-
-    /**
-     * Initialize a scanner over the given input split using this task attempt configuration.
-     *
-     * @deprecated since 1.7.0; Use {@link #contextIterators} instead.
-     */
-    @Deprecated(since = "1.7.0")
-    protected void setupIterators(TaskAttemptContext context, Scanner scanner,
-        org.apache.accumulo.core.client.mapreduce.RangeInputSplit split) {
-      setupIterators(context, scanner, null, split);
-    }
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputTableConfig.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputTableConfig.java
deleted file mode 100644
index c48df18..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputTableConfig.java
+++ /dev/null
@@ -1,394 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.client.mapreduce;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Objects;
-
-import org.apache.accumulo.core.client.ClientSideIteratorScanner;
-import org.apache.accumulo.core.client.IsolatedScanner;
-import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.ScannerBase;
-import org.apache.accumulo.core.client.sample.SamplerConfiguration;
-import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
-import org.apache.accumulo.core.util.Pair;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-
-/**
- * This class to holds a batch scan configuration for a table. It contains all the properties needed
- * to specify how rows should be returned from the table.
- *
- * @deprecated since 2.0.0; Use org.apache.accumulo.hadoop.mapreduce instead from the
- *             accumulo-hadoop-mapreduce.jar
- */
-@Deprecated(since = "2.0.0")
-public class InputTableConfig implements Writable {
-
-  private List<IteratorSetting> iterators;
-  private List<Range> ranges;
-  private Collection<Pair<Text,Text>> columns;
-
-  private boolean autoAdjustRanges = true;
-  private boolean useLocalIterators = false;
-  private boolean useIsolatedScanners = false;
-  private boolean offlineScan = false;
-  private SamplerConfiguration samplerConfig = null;
-
-  public InputTableConfig() {}
-
-  /**
-   * Creates a batch scan config object out of a previously serialized batch scan config object.
-   *
-   * @param input the data input of the serialized batch scan config
-   */
-  public InputTableConfig(DataInput input) throws IOException {
-    readFields(input);
-  }
-
-  /**
-   * Sets the input ranges to scan for all tables associated with this job. This will be added to
-   * any per-table ranges that have been set using
-   *
-   * @param ranges the ranges that will be mapped over
-   * @since 1.6.0
-   */
-  public InputTableConfig setRanges(List<Range> ranges) {
-    this.ranges = ranges;
-    return this;
-  }
-
-  /**
-   * Returns the ranges to be queried in the configuration
-   */
-  public List<Range> getRanges() {
-    return ranges != null ? ranges : new ArrayList<>();
-  }
-
-  /**
-   * Restricts the columns that will be mapped over for this job for the default input table.
-   *
-   * @param columns a pair of {@link Text} objects corresponding to column family and column
-   *        qualifier. If the column qualifier is null, the entire column family is selected. An
-   *        empty set is the default and is equivalent to scanning the all columns.
-   * @since 1.6.0
-   */
-  public InputTableConfig fetchColumns(Collection<Pair<Text,Text>> columns) {
-    this.columns = columns;
-    return this;
-  }
-
-  /**
-   * Returns the columns to be fetched for this configuration
-   */
-  public Collection<Pair<Text,Text>> getFetchedColumns() {
-    return columns != null ? columns : new HashSet<>();
-  }
-
-  /**
-   * Set iterators on to be used in the query.
-   *
-   * @param iterators the configurations for the iterators
-   * @since 1.6.0
-   */
-  public InputTableConfig setIterators(List<IteratorSetting> iterators) {
-    this.iterators = iterators;
-    return this;
-  }
-
-  /**
-   * Returns the iterators to be set on this configuration
-   */
-  public List<IteratorSetting> getIterators() {
-    return iterators != null ? iterators : new ArrayList<>();
-  }
-
-  /**
-   * Controls the automatic adjustment of ranges for this job. This feature merges overlapping
-   * ranges, then splits them to align with tablet boundaries. Disabling this feature will cause
-   * exactly one Map task to be created for each specified range. The default setting is enabled. *
-   *
-   * <p>
-   * By default, this feature is <b>enabled</b>.
-   *
-   * @param autoAdjustRanges the feature is enabled if true, disabled otherwise
-   * @see #setRanges(java.util.List)
-   * @since 1.6.0
-   */
-  public InputTableConfig setAutoAdjustRanges(boolean autoAdjustRanges) {
-    this.autoAdjustRanges = autoAdjustRanges;
-    return this;
-  }
-
-  /**
-   * Determines whether a configuration has auto-adjust ranges enabled.
-   *
-   * @return false if the feature is disabled, true otherwise
-   * @since 1.6.0
-   * @see #setAutoAdjustRanges(boolean)
-   */
-  public boolean shouldAutoAdjustRanges() {
-    return autoAdjustRanges;
-  }
-
-  /**
-   * Controls the use of the {@link ClientSideIteratorScanner} in this job. Enabling this feature
-   * will cause the iterator stack to be constructed within the Map task, rather than within the
-   * Accumulo TServer. To use this feature, all classes needed for those iterators must be available
-   * on the classpath for the task.
-   *
-   * <p>
-   * By default, this feature is <b>disabled</b>.
-   *
-   * @param useLocalIterators the feature is enabled if true, disabled otherwise
-   * @since 1.6.0
-   */
-  public InputTableConfig setUseLocalIterators(boolean useLocalIterators) {
-    this.useLocalIterators = useLocalIterators;
-    return this;
-  }
-
-  /**
-   * Determines whether a configuration uses local iterators.
-   *
-   * @return true if the feature is enabled, false otherwise
-   * @since 1.6.0
-   * @see #setUseLocalIterators(boolean)
-   */
-  public boolean shouldUseLocalIterators() {
-    return useLocalIterators;
-  }
-
-  /**
-   * Enable reading offline tables. By default, this feature is disabled and only online tables are
-   * scanned. This will make the map reduce job directly read the table's files. If the table is not
-   * offline, then the job will fail. If the table comes online during the map reduce job, it is
-   * likely that the job will fail.
-   *
-   * <p>
-   * To use this option, the map reduce user will need access to read the Accumulo directory in
-   * HDFS.
-   *
-   * <p>
-   * Reading the offline table will create the scan time iterator stack in the map process. So any
-   * iterators that are configured for the table will need to be on the mapper's classpath. The
-   * accumulo.properties may need to be on the mapper's classpath if HDFS or the Accumulo directory
-   * in HDFS are non-standard.
-   *
-   * <p>
-   * One way to use this feature is to clone a table, take the clone offline, and use the clone as
-   * the input table for a map reduce job. If you plan to map reduce over the data many times, it
-   * may be better to the compact the table, clone it, take it offline, and use the clone for all
-   * map reduce jobs. The reason to do this is that compaction will reduce each tablet in the table
-   * to one file, and it is faster to read from one file.
-   *
-   * <p>
-   * There are two possible advantages to reading a tables file directly out of HDFS. First, you may
-   * see better read performance. Second, it will support speculative execution better. When reading
-   * an online table speculative execution can put more load on an already slow tablet server.
-   *
-   * <p>
-   * By default, this feature is <b>disabled</b>.
-   *
-   * @param offlineScan the feature is enabled if true, disabled otherwise
-   * @since 1.6.0
-   */
-  public InputTableConfig setOfflineScan(boolean offlineScan) {
-    this.offlineScan = offlineScan;
-    return this;
-  }
-
-  /**
-   * Determines whether a configuration has the offline table scan feature enabled.
-   *
-   * @return true if the feature is enabled, false otherwise
-   * @since 1.6.0
-   * @see #setOfflineScan(boolean)
-   */
-  public boolean isOfflineScan() {
-    return offlineScan;
-  }
-
-  /**
-   * Controls the use of the {@link IsolatedScanner} in this job.
-   *
-   * <p>
-   * By default, this feature is <b>disabled</b>.
-   *
-   * @param useIsolatedScanners the feature is enabled if true, disabled otherwise
-   * @since 1.6.0
-   */
-  public InputTableConfig setUseIsolatedScanners(boolean useIsolatedScanners) {
-    this.useIsolatedScanners = useIsolatedScanners;
-    return this;
-  }
-
-  /**
-   * Determines whether a configuration has isolation enabled.
-   *
-   * @return true if the feature is enabled, false otherwise
-   * @since 1.6.0
-   * @see #setUseIsolatedScanners(boolean)
-   */
-  public boolean shouldUseIsolatedScanners() {
-    return useIsolatedScanners;
-  }
-
-  /**
-   * Set the sampler configuration to use when reading from the data.
-   *
-   * @see ScannerBase#setSamplerConfiguration(SamplerConfiguration)
-   * @see InputFormatBase#setSamplerConfiguration(org.apache.hadoop.mapreduce.Job,
-   *      SamplerConfiguration)
-   *
-   * @since 1.8.0
-   */
-  public void setSamplerConfiguration(SamplerConfiguration samplerConfiguration) {
-    this.samplerConfig = samplerConfiguration;
-  }
-
-  /**
-   *
-   * @since 1.8.0
-   */
-  public SamplerConfiguration getSamplerConfiguration() {
-    return samplerConfig;
-  }
-
-  @Override
-  public void write(DataOutput dataOutput) throws IOException {
-    if (iterators != null) {
-      dataOutput.writeInt(iterators.size());
-      for (IteratorSetting setting : iterators) {
-        setting.write(dataOutput);
-      }
-    } else {
-      dataOutput.writeInt(0);
-    }
-    if (ranges != null) {
-      dataOutput.writeInt(ranges.size());
-      for (Range range : ranges) {
-        range.write(dataOutput);
-      }
-    } else {
-      dataOutput.writeInt(0);
-    }
-    if (columns != null) {
-      dataOutput.writeInt(columns.size());
-      for (Pair<Text,Text> column : columns) {
-        if (column.getSecond() == null) {
-          dataOutput.writeInt(1);
-          column.getFirst().write(dataOutput);
-        } else {
-          dataOutput.writeInt(2);
-          column.getFirst().write(dataOutput);
-          column.getSecond().write(dataOutput);
-        }
-      }
-    } else {
-      dataOutput.writeInt(0);
-    }
-    dataOutput.writeBoolean(autoAdjustRanges);
-    dataOutput.writeBoolean(useLocalIterators);
-    dataOutput.writeBoolean(useIsolatedScanners);
-    dataOutput.writeBoolean(offlineScan);
-    if (samplerConfig == null) {
-      dataOutput.writeBoolean(false);
-    } else {
-      dataOutput.writeBoolean(true);
-      new SamplerConfigurationImpl(samplerConfig).write(dataOutput);
-    }
-  }
-
-  @Override
-  public void readFields(DataInput dataInput) throws IOException {
-    // load iterators
-    long iterSize = dataInput.readInt();
-    if (iterSize > 0) {
-      iterators = new ArrayList<>();
-    }
-    for (int i = 0; i < iterSize; i++) {
-      iterators.add(new IteratorSetting(dataInput));
-    }
-    // load ranges
-    long rangeSize = dataInput.readInt();
-    if (rangeSize > 0) {
-      ranges = new ArrayList<>();
-    }
-    for (int i = 0; i < rangeSize; i++) {
-      Range range = new Range();
-      range.readFields(dataInput);
-      ranges.add(range);
-    }
-    // load columns
-    long columnSize = dataInput.readInt();
-    if (columnSize > 0) {
-      columns = new HashSet<>();
-    }
-    for (int i = 0; i < columnSize; i++) {
-      long numPairs = dataInput.readInt();
-      Text colFam = new Text();
-      colFam.readFields(dataInput);
-      if (numPairs == 1) {
-        columns.add(new Pair<>(colFam, null));
-      } else if (numPairs == 2) {
-        Text colQual = new Text();
-        colQual.readFields(dataInput);
-        columns.add(new Pair<>(colFam, colQual));
-      }
-    }
-    autoAdjustRanges = dataInput.readBoolean();
-    useLocalIterators = dataInput.readBoolean();
-    useIsolatedScanners = dataInput.readBoolean();
-    offlineScan = dataInput.readBoolean();
-
-    if (dataInput.readBoolean()) {
-      samplerConfig = new SamplerConfigurationImpl(dataInput).toSamplerConfiguration();
-    }
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) {
-      return true;
-    }
-    if (o == null || getClass() != o.getClass()) {
-      return false;
-    }
-    InputTableConfig that = (InputTableConfig) o;
-    return autoAdjustRanges == that.autoAdjustRanges && useLocalIterators == that.useLocalIterators
-        && useIsolatedScanners == that.useIsolatedScanners && offlineScan == that.offlineScan
-        && Objects.equals(iterators, that.iterators) && Objects.equals(ranges, that.ranges)
-        && Objects.equals(columns, that.columns)
-        && Objects.equals(samplerConfig, that.samplerConfig);
-  }
-
-  @Override
-  public int hashCode() {
-    return Objects.hash(iterators, ranges, columns, autoAdjustRanges, useLocalIterators,
-        useIsolatedScanners, offlineScan, samplerConfig);
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
deleted file mode 100644
index dcdf837..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
+++ /dev/null
@@ -1,524 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.client.mapreduce;
-
-import static java.nio.charset.StandardCharsets.UTF_8;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Base64;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.sample.SamplerConfiguration;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken.AuthenticationTokenSerializer;
-import org.apache.accumulo.core.data.ByteSequence;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.PartialKey;
-import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.Pair;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.log4j.Level;
-
-/**
- * The Class RangeInputSplit. Encapsulates an Accumulo range for use in Map Reduce jobs.
- *
- * @deprecated since 2.0.0; Use org.apache.accumulo.hadoop.mapreduce instead from the
- *             accumulo-hadoop-mapreduce.jar
- */
-@Deprecated(since = "2.0.0")
-public class RangeInputSplit extends InputSplit implements Writable {
-  private Range range;
-  private String[] locations;
-  private String tableId, tableName, instanceName, zooKeepers, principal;
-  private org.apache.accumulo.core.clientImpl.mapreduce.lib.ConfiguratorBase.TokenSource tokenSource;
-  private String tokenFile;
-  private AuthenticationToken token;
-  private Boolean offline, isolatedScan, localIterators;
-  private Authorizations auths;
-  private Set<Pair<Text,Text>> fetchedColumns;
-  private List<IteratorSetting> iterators;
-  private SamplerConfiguration samplerConfig;
-  private Level level;
-
-  public RangeInputSplit() {
-    range = new Range();
-    locations = new String[0];
-    tableName = "";
-    tableId = "";
-  }
-
-  public RangeInputSplit(RangeInputSplit split) throws IOException {
-    this.range = split.getRange();
-    this.setLocations(split.getLocations());
-    this.setTableName(split.getTableName());
-    this.setTableId(split.getTableId());
-  }
-
-  protected RangeInputSplit(String table, String tableId, Range range, String[] locations) {
-    this.range = range;
-    setLocations(locations);
-    this.tableName = table;
-    this.tableId = tableId;
-  }
-
-  public Range getRange() {
-    return range;
-  }
-
-  public static float getProgress(ByteSequence start, ByteSequence end, ByteSequence position) {
-    return org.apache.accumulo.core.clientImpl.mapreduce.SplitUtils.getProgress(start, end,
-        position);
-  }
-
-  public float getProgress(Key currentKey) {
-    if (currentKey == null) {
-      return 0f;
-    }
-    if (range.contains(currentKey)) {
-      if (range.getStartKey() != null && range.getEndKey() != null) {
-        if (range.getStartKey().compareTo(range.getEndKey(), PartialKey.ROW) != 0) {
-          // just look at the row progress
-          return getProgress(range.getStartKey().getRowData(), range.getEndKey().getRowData(),
-              currentKey.getRowData());
-        } else if (range.getStartKey().compareTo(range.getEndKey(), PartialKey.ROW_COLFAM) != 0) {
-          // just look at the column family progress
-          return getProgress(range.getStartKey().getColumnFamilyData(),
-              range.getEndKey().getColumnFamilyData(), currentKey.getColumnFamilyData());
-        } else if (range.getStartKey().compareTo(range.getEndKey(), PartialKey.ROW_COLFAM_COLQUAL)
-            != 0) {
-          // just look at the column qualifier progress
-          return getProgress(range.getStartKey().getColumnQualifierData(),
-              range.getEndKey().getColumnQualifierData(), currentKey.getColumnQualifierData());
-        }
-      }
-    }
-    // if we can't figure it out, then claim no progress
-    return 0f;
-  }
-
-  /**
-   * This implementation of length is only an estimate, it does not provide exact values. Do not
-   * have your code rely on this return value.
-   */
-  @Override
-  public long getLength() throws IOException {
-    return org.apache.accumulo.core.clientImpl.mapreduce.SplitUtils.getRangeLength(range);
-  }
-
-  @Override
-  public String[] getLocations() throws IOException {
-    return Arrays.copyOf(locations, locations.length);
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    range.readFields(in);
-    tableName = in.readUTF();
-    tableId = in.readUTF();
-    int numLocs = in.readInt();
-    locations = new String[numLocs];
-    for (int i = 0; i < numLocs; ++i) {
-      locations[i] = in.readUTF();
-    }
-
-    if (in.readBoolean()) {
-      isolatedScan = in.readBoolean();
-    }
-
-    if (in.readBoolean()) {
-      offline = in.readBoolean();
-    }
-
-    if (in.readBoolean()) {
-      localIterators = in.readBoolean();
-    }
-
-    if (in.readBoolean()) {
-      int numColumns = in.readInt();
-      List<String> columns = new ArrayList<>(numColumns);
-      for (int i = 0; i < numColumns; i++) {
-        columns.add(in.readUTF());
-      }
-
-      fetchedColumns = org.apache.accumulo.core.clientImpl.mapreduce.lib.InputConfigurator
-          .deserializeFetchedColumns(columns);
-    }
-
-    if (in.readBoolean()) {
-      String strAuths = in.readUTF();
-      auths = new Authorizations(strAuths.getBytes(UTF_8));
-    }
-
-    if (in.readBoolean()) {
-      principal = in.readUTF();
-    }
-
-    if (in.readBoolean()) {
-      int ordinal = in.readInt();
-      this.tokenSource =
-          org.apache.accumulo.core.clientImpl.mapreduce.lib.ConfiguratorBase.TokenSource
-              .values()[ordinal];
-
-      switch (this.tokenSource) {
-        case INLINE:
-          String tokenClass = in.readUTF();
-          byte[] base64TokenBytes = in.readUTF().getBytes(UTF_8);
-          byte[] tokenBytes = Base64.getDecoder().decode(base64TokenBytes);
-
-          this.token = AuthenticationTokenSerializer.deserialize(tokenClass, tokenBytes);
-          break;
-
-        case FILE:
-          this.tokenFile = in.readUTF();
-
-          break;
-        default:
-          throw new IOException("Cannot parse unknown TokenSource ordinal");
-      }
-    }
-
-    if (in.readBoolean()) {
-      instanceName = in.readUTF();
-    }
-
-    if (in.readBoolean()) {
-      zooKeepers = in.readUTF();
-    }
-
-    if (in.readBoolean()) {
-      int numIterators = in.readInt();
-      iterators = new ArrayList<>(numIterators);
-      for (int i = 0; i < numIterators; i++) {
-        iterators.add(new IteratorSetting(in));
-      }
-    }
-
-    if (in.readBoolean()) {
-      level = Level.toLevel(in.readInt());
-    }
-
-    if (in.readBoolean()) {
-      samplerConfig = new SamplerConfigurationImpl(in).toSamplerConfiguration();
-    }
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    range.write(out);
-    out.writeUTF(tableName);
-    out.writeUTF(tableId);
-    out.writeInt(locations.length);
-    for (String location : locations) {
-      out.writeUTF(location);
-    }
-
-    out.writeBoolean(isolatedScan != null);
-    if (isolatedScan != null) {
-      out.writeBoolean(isolatedScan);
-    }
-
-    out.writeBoolean(offline != null);
-    if (offline != null) {
-      out.writeBoolean(offline);
-    }
-
-    out.writeBoolean(localIterators != null);
-    if (localIterators != null) {
-      out.writeBoolean(localIterators);
-    }
-
-    out.writeBoolean(fetchedColumns != null);
-    if (fetchedColumns != null) {
-      String[] cols = org.apache.accumulo.core.clientImpl.mapreduce.lib.InputConfigurator
-          .serializeColumns(fetchedColumns);
-      out.writeInt(cols.length);
-      for (String col : cols) {
-        out.writeUTF(col);
-      }
-    }
-
-    out.writeBoolean(auths != null);
-    if (auths != null) {
-      out.writeUTF(auths.serialize());
-    }
-
-    out.writeBoolean(principal != null);
-    if (principal != null) {
-      out.writeUTF(principal);
-    }
-
-    out.writeBoolean(tokenSource != null);
-    if (tokenSource != null) {
-      out.writeInt(tokenSource.ordinal());
-
-      if (token != null && tokenFile != null) {
-        throw new IOException(
-            "Cannot use both inline AuthenticationToken and file-based AuthenticationToken");
-      } else if (token != null) {
-        out.writeUTF(token.getClass().getName());
-        out.writeUTF(
-            Base64.getEncoder().encodeToString(AuthenticationTokenSerializer.serialize(token)));
-      } else {
-        out.writeUTF(tokenFile);
-      }
-    }
-
-    out.writeBoolean(instanceName != null);
-    if (instanceName != null) {
-      out.writeUTF(instanceName);
-    }
-
-    out.writeBoolean(zooKeepers != null);
-    if (zooKeepers != null) {
-      out.writeUTF(zooKeepers);
-    }
-
-    out.writeBoolean(iterators != null);
-    if (iterators != null) {
-      out.writeInt(iterators.size());
-      for (IteratorSetting iterator : iterators) {
-        iterator.write(out);
-      }
-    }
-
-    out.writeBoolean(level != null);
-    if (level != null) {
-      out.writeInt(level.toInt());
-    }
-
-    out.writeBoolean(samplerConfig != null);
-    if (samplerConfig != null) {
-      new SamplerConfigurationImpl(samplerConfig).write(out);
-    }
-  }
-
-  /**
-   * Use {@link #getTableName}
-   *
-   * @deprecated since 1.6.1, use getTableName() instead.
-   */
-  @Deprecated(since = "1.6.1")
-  public String getTable() {
-    return getTableName();
-  }
-
-  public String getTableName() {
-    return tableName;
-  }
-
-  /**
-   * Use {@link #setTableName}
-   *
-   * @deprecated since 1.6.1, use setTableName() instead.
-   */
-  @Deprecated(since = "1.6.1")
-  public void setTable(String table) {
-    setTableName(table);
-  }
-
-  public void setTableName(String table) {
-    this.tableName = table;
-  }
-
-  public void setTableId(String tableId) {
-    this.tableId = tableId;
-  }
-
-  public String getTableId() {
-    return tableId;
-  }
-
-  /**
-   * @see #getInstance(org.apache.accumulo.core.client.ClientConfiguration)
-   * @deprecated since 1.7.0, use getInstance(ClientConfiguration) instead.
-   */
-  @Deprecated(since = "1.7.0")
-  public org.apache.accumulo.core.client.Instance getInstance() {
-    return getInstance(org.apache.accumulo.core.client.ClientConfiguration.loadDefault());
-  }
-
-  public org.apache.accumulo.core.client.Instance
-      getInstance(org.apache.accumulo.core.client.ClientConfiguration base) {
-    if (null == instanceName) {
-      return null;
-    }
-
-    if (null == zooKeepers) {
-      return null;
-    }
-
-    return new org.apache.accumulo.core.client.ZooKeeperInstance(
-        base.withInstance(getInstanceName()).withZkHosts(getZooKeepers()));
-  }
-
-  public String getInstanceName() {
-    return instanceName;
-  }
-
-  public void setInstanceName(String instanceName) {
-    this.instanceName = instanceName;
-  }
-
-  public String getZooKeepers() {
-    return zooKeepers;
-  }
-
-  public void setZooKeepers(String zooKeepers) {
-    this.zooKeepers = zooKeepers;
-  }
-
-  public String getPrincipal() {
-    return principal;
-  }
-
-  public void setPrincipal(String principal) {
-    this.principal = principal;
-  }
-
-  public AuthenticationToken getToken() {
-    return token;
-  }
-
-  public void setToken(AuthenticationToken token) {
-    this.tokenSource =
-        org.apache.accumulo.core.clientImpl.mapreduce.lib.ConfiguratorBase.TokenSource.INLINE;
-    this.token = token;
-  }
-
-  public void setToken(String tokenFile) {
-    this.tokenSource =
-        org.apache.accumulo.core.clientImpl.mapreduce.lib.ConfiguratorBase.TokenSource.FILE;
-    this.tokenFile = tokenFile;
-  }
-
-  public Boolean isOffline() {
-    return offline;
-  }
-
-  public void setOffline(Boolean offline) {
-    this.offline = offline;
-  }
-
-  public void setLocations(String[] locations) {
-    this.locations = Arrays.copyOf(locations, locations.length);
-  }
-
-  public Boolean isIsolatedScan() {
-    return isolatedScan;
-  }
-
-  public void setIsolatedScan(Boolean isolatedScan) {
-    this.isolatedScan = isolatedScan;
-  }
-
-  public Authorizations getAuths() {
-    return auths;
-  }
-
-  public void setAuths(Authorizations auths) {
-    this.auths = auths;
-  }
-
-  public void setRange(Range range) {
-    this.range = range;
-  }
-
-  public Boolean usesLocalIterators() {
-    return localIterators;
-  }
-
-  public void setUsesLocalIterators(Boolean localIterators) {
-    this.localIterators = localIterators;
-  }
-
-  public Set<Pair<Text,Text>> getFetchedColumns() {
-    return fetchedColumns;
-  }
-
-  public void setFetchedColumns(Collection<Pair<Text,Text>> fetchedColumns) {
-    this.fetchedColumns = new HashSet<>();
-    for (Pair<Text,Text> columns : fetchedColumns) {
-      this.fetchedColumns.add(columns);
-    }
-  }
-
-  public void setFetchedColumns(Set<Pair<Text,Text>> fetchedColumns) {
-    this.fetchedColumns = fetchedColumns;
-  }
-
-  public List<IteratorSetting> getIterators() {
-    return iterators;
-  }
-
-  public void setIterators(List<IteratorSetting> iterators) {
-    this.iterators = iterators;
-  }
-
-  public Level getLogLevel() {
-    return level;
-  }
-
-  public void setLogLevel(Level level) {
-    this.level = level;
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder sb = new StringBuilder(256);
-    sb.append("Range: ").append(range);
-    sb.append(" Locations: ").append(Arrays.asList(locations));
-    sb.append(" Table: ").append(tableName);
-    sb.append(" TableID: ").append(tableId);
-    sb.append(" InstanceName: ").append(instanceName);
-    sb.append(" zooKeepers: ").append(zooKeepers);
-    sb.append(" principal: ").append(principal);
-    sb.append(" tokenSource: ").append(tokenSource);
-    sb.append(" authenticationToken: ").append(token);
-    sb.append(" authenticationTokenFile: ").append(tokenFile);
-    sb.append(" Authorizations: ").append(auths);
-    sb.append(" offlineScan: ").append(offline);
-    sb.append(" isolatedScan: ").append(isolatedScan);
-    sb.append(" localIterators: ").append(localIterators);
-    sb.append(" fetchColumns: ").append(fetchedColumns);
-    sb.append(" iterators: ").append(iterators);
-    sb.append(" logLevel: ").append(level);
-    sb.append(" samplerConfig: ").append(samplerConfig);
-    return sb.toString();
-  }
-
-  public void setSamplerConfiguration(SamplerConfiguration samplerConfiguration) {
-    this.samplerConfig = samplerConfiguration;
-  }
-
-  public SamplerConfiguration getSamplerConfiguration() {
-    return samplerConfig;
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/partition/KeyRangePartitioner.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/partition/KeyRangePartitioner.java
deleted file mode 100644
index fea37ae..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/partition/KeyRangePartitioner.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.client.mapreduce.lib.partition;
-
-import org.apache.accumulo.core.data.Key;
-import org.apache.hadoop.conf.Configurable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.Partitioner;
-
-/**
- * Hadoop partitioner that uses ranges based on row keys, and optionally sub-bins based on hashing.
- *
- * @deprecated since 2.0.0; Use org.apache.accumulo.hadoop.mapreduce.partition instead from the
- *             accumulo-hadoop-mapreduce.jar
- */
-@Deprecated(since = "2.0.0")
-public class KeyRangePartitioner extends Partitioner<Key,Writable> implements Configurable {
-  private RangePartitioner rp = new RangePartitioner();
-
-  @Override
-  public int getPartition(Key key, Writable value, int numPartitions) {
-    return rp.getPartition(key.getRow(), value, numPartitions);
-  }
-
-  @Override
-  public Configuration getConf() {
-    return rp.getConf();
-  }
-
-  @Override
-  public void setConf(Configuration conf) {
-    rp.setConf(conf);
-  }
-
-  /**
-   * Sets the hdfs file name to use, containing a newline separated list of Base64 encoded split
-   * points that represent ranges for partitioning
-   */
-  public static void setSplitFile(Job job, String file) {
-    RangePartitioner.setSplitFile(job, file);
-  }
-
-  /**
-   * Sets the number of random sub-bins per range
-   */
-  public static void setNumSubBins(Job job, int num) {
-    RangePartitioner.setNumSubBins(job, num);
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/partition/RangePartitioner.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/partition/RangePartitioner.java
deleted file mode 100644
index 91ccf3c..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/partition/RangePartitioner.java
+++ /dev/null
@@ -1,133 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.client.mapreduce.lib.partition;
-
-import static java.nio.charset.StandardCharsets.UTF_8;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.Arrays;
-import java.util.Base64;
-import java.util.Scanner;
-import java.util.TreeSet;
-
-import org.apache.accumulo.core.clientImpl.mapreduce.lib.DistributedCacheHelper;
-import org.apache.hadoop.conf.Configurable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.Partitioner;
-
-/**
- * Hadoop partitioner that uses ranges, and optionally sub-bins based on hashing.
- *
- * @deprecated since 2.0.0; Use org.apache.accumulo.hadoop.mapreduce.partition instead from the
- *             accumulo-hadoop-mapreduce.jar
- */
-@Deprecated(since = "2.0.0")
-public class RangePartitioner extends Partitioner<Text,Writable> implements Configurable {
-  private static final String PREFIX = RangePartitioner.class.getName();
-  private static final String CUTFILE_KEY = PREFIX + ".cutFile";
-  private static final String NUM_SUBBINS = PREFIX + ".subBins";
-
-  private Configuration conf;
-
-  @Override
-  public int getPartition(Text key, Writable value, int numPartitions) {
-    try {
-      return findPartition(key, getCutPoints(), getNumSubBins());
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  int findPartition(Text key, Text[] array, int numSubBins) {
-    // find the bin for the range, and guarantee it is positive
-    int index = Arrays.binarySearch(array, key);
-    index = index < 0 ? (index + 1) * -1 : index;
-
-    // both conditions work with numSubBins == 1, but this check is to avoid
-    // hashing, when we don't need to, for speed
-    if (numSubBins < 2) {
-      return index;
-    }
-    return (key.toString().hashCode() & Integer.MAX_VALUE) % numSubBins + index * numSubBins;
-  }
-
-  private int _numSubBins = 0;
-
-  private synchronized int getNumSubBins() {
-    if (_numSubBins < 1) {
-      // get number of sub-bins and guarantee it is positive
-      _numSubBins = Math.max(1, getConf().getInt(NUM_SUBBINS, 1));
-    }
-    return _numSubBins;
-  }
-
-  private Text[] cutPointArray = null;
-
-  private synchronized Text[] getCutPoints() throws IOException {
-    if (cutPointArray == null) {
-      String cutFileName = conf.get(CUTFILE_KEY);
-      TreeSet<Text> cutPoints = new TreeSet<>();
-      try (
-          InputStream inputStream =
-              DistributedCacheHelper.openCachedFile(cutFileName, CUTFILE_KEY, conf);
-          Scanner in = new Scanner(inputStream, UTF_8)) {
-        while (in.hasNextLine()) {
-          cutPoints.add(new Text(Base64.getDecoder().decode(in.nextLine())));
-        }
-      }
-
-      cutPointArray = cutPoints.toArray(new Text[cutPoints.size()]);
-
-      if (cutPointArray == null) {
-        throw new IOException("Cutpoint array not properly created from file" + cutFileName);
-      }
-    }
-    return cutPointArray;
-  }
-
-  @Override
-  public Configuration getConf() {
-    return conf;
-  }
-
-  @Override
-  public void setConf(Configuration conf) {
-    this.conf = conf;
-  }
-
-  /**
-   * Sets the hdfs file name to use, containing a newline separated list of Base64 encoded split
-   * points that represent ranges for partitioning
-   */
-  public static void setSplitFile(Job job, String file) {
-    DistributedCacheHelper.addCacheFile(job, file, CUTFILE_KEY);
-    job.getConfiguration().set(CUTFILE_KEY, file);
-  }
-
-  /**
-   * Sets the number of random sub-bins per range
-   */
-  public static void setNumSubBins(Job job, int num) {
-    job.getConfiguration().setInt(NUM_SUBBINS, num);
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/replication/PeerExistsException.java b/core/src/main/java/org/apache/accumulo/core/client/replication/PeerExistsException.java
deleted file mode 100644
index 34b3052..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/replication/PeerExistsException.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.client.replication;
-
-/**
- * The peer already exists
- */
-@Deprecated(since = "2.1.0")
-public class PeerExistsException extends Exception {
-  private static final long serialVersionUID = 1L;
-
-  public PeerExistsException(String peer) {
-    this(peer, (String) null);
-  }
-
-  public PeerExistsException(String peer, String message) {
-    super("Peer '" + peer + "' already exists"
-        + (message == null || message.isEmpty() ? "" : message));
-  }
-
-  public PeerExistsException(String message, Throwable cause) {
-    super(message, cause);
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/replication/PeerNotFoundException.java b/core/src/main/java/org/apache/accumulo/core/client/replication/PeerNotFoundException.java
deleted file mode 100644
index 4da5ec9..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/replication/PeerNotFoundException.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.client.replication;
-
-/**
- * The peer already exists
- */
-@Deprecated(since = "2.1.0")
-public class PeerNotFoundException extends Exception {
-  private static final long serialVersionUID = 1L;
-
-  public PeerNotFoundException(String peer) {
-    this(peer, (String) null);
-  }
-
-  public PeerNotFoundException(String peer, String message) {
-    super("Peer '" + peer + "' not found " + (message == null || message.isEmpty() ? "" : message));
-  }
-
-  public PeerNotFoundException(String message, Throwable cause) {
-    super(message, cause);
-  }
-
-  public PeerNotFoundException(String peer, String message, Throwable cause) {
-    super("Peer '" + peer + "' not found " + message, cause);
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/rfile/RFile.java b/core/src/main/java/org/apache/accumulo/core/client/rfile/RFile.java
index 3b6d10a..64956dc 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/rfile/RFile.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/rfile/RFile.java
@@ -23,6 +23,7 @@
 import java.util.Collection;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Objects;
 import java.util.function.Predicate;
 
 import org.apache.accumulo.core.client.Scanner;
@@ -36,7 +37,9 @@
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.util.RowRangeUtil;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 
 /**
@@ -76,6 +79,38 @@
      * @return this
      */
     ScannerFSOptions from(String... files);
+
+    /**
+     * Specify FencedPath files to read from. When multiple are specified the {@link Scanner}
+     * constructed will present a merged view.
+     *
+     * @param files one or more FencedPaths to read.
+     * @return this
+     *
+     * @since 3.1.0
+     */
+    ScannerFSOptions from(FencedPath... files);
+
+    /**
+     * @since 3.1.0
+     */
+    class FencedPath {
+      private final Path path;
+      private final Range fence;
+
+      public FencedPath(Path path, Range fence) {
+        this.path = Objects.requireNonNull(path);
+        this.fence = RowRangeUtil.requireKeyExtentDataRange(fence);
+      }
+
+      public Path getPath() {
+        return path;
+      }
+
+      public Range getFence() {
+        return fence;
+      }
+    }
   }
 
   /**
diff --git a/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileScanner.java b/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileScanner.java
index 767ebc3..b62fc84 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileScanner.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileScanner.java
@@ -19,6 +19,7 @@
 package org.apache.accumulo.core.client.rfile;
 
 import java.io.IOException;
+import java.io.UncheckedIOException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
@@ -140,6 +141,11 @@
         public long requestCount() {
           return 0L;
         }
+
+        @Override
+        public long evictionCount() {
+          return 0L;
+        }
       };
     }
 
@@ -211,10 +217,9 @@
         blockCacheManager.start(BlockCacheConfiguration.forTabletServer(cc));
         this.indexCache = blockCacheManager.getBlockCache(CacheType.INDEX);
         this.dataCache = blockCacheManager.getBlockCache(CacheType.DATA);
-      } catch (RuntimeException e) {
-        throw e;
-      } catch (Exception e) {
-        throw new RuntimeException(e);
+      } catch (ReflectiveOperationException e) {
+        throw new IllegalArgumentException(
+            "Configuration does not contain loadable class for block cache manager factory", e);
       }
     }
     if (indexCache == null) {
@@ -351,7 +356,7 @@
         CachableBuilder cb =
             new CachableBuilder().input(inputStream, "source-" + i).length(sources[i].getLength())
                 .conf(opts.in.getConf()).cacheProvider(cacheProvider).cryptoService(cryptoService);
-        readers.add(new RFile.Reader(cb));
+        readers.add(RFile.getReader(cb, sources[i].getRange()));
       }
 
       if (getSamplerConfiguration() != null) {
@@ -389,14 +394,14 @@
           iterator = IteratorConfigUtil.loadIterators(iterator, iteratorBuilder);
         }
       } catch (IOException e) {
-        throw new RuntimeException(e);
+        throw new UncheckedIOException(e);
       }
 
       iterator.seek(getRange() == null ? EMPTY_RANGE : getRange(), families, !families.isEmpty());
       return new IteratorAdapter(iterator);
 
     } catch (IOException e) {
-      throw new RuntimeException(e);
+      throw new UncheckedIOException(e);
     }
   }
 
@@ -407,14 +412,10 @@
         source.getInputStream().close();
       }
     } catch (IOException e) {
-      throw new RuntimeException(e);
+      throw new UncheckedIOException(e);
     }
-    try {
-      if (this.blockCacheManager != null) {
-        this.blockCacheManager.stop();
-      }
-    } catch (Exception e1) {
-      throw new RuntimeException(e1);
+    if (this.blockCacheManager != null) {
+      this.blockCacheManager.stop();
     }
   }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileScannerBuilder.java b/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileScannerBuilder.java
index f4ab7e9..e63c795 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileScannerBuilder.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileScannerBuilder.java
@@ -38,32 +38,37 @@
 class RFileScannerBuilder implements RFile.InputArguments, RFile.ScannerFSOptions {
 
   static class InputArgs extends FSConfArgs {
-    private Path[] paths;
+    private FencedPath[] rFiles;
     private RFileSource[] sources;
 
     InputArgs(String... files) {
-      this.paths = new Path[files.length];
+      this.rFiles = new FencedPath[files.length];
       for (int i = 0; i < files.length; i++) {
-        this.paths[i] = new Path(files[i]);
+        this.rFiles[i] = new FencedPath(new Path(files[i]), new Range());
       }
     }
 
+    InputArgs(FencedPath... files) {
+      this.rFiles = files;
+    }
+
     InputArgs(RFileSource... sources) {
       this.sources = sources;
     }
 
     RFileSource[] getSources() throws IOException {
       if (sources == null) {
-        sources = new RFileSource[paths.length];
-        for (int i = 0; i < paths.length; i++) {
-          sources[i] = new RFileSource(getFileSystem().open(paths[i]),
-              getFileSystem().getFileStatus(paths[i]).getLen());
+        sources = new RFileSource[rFiles.length];
+        for (int i = 0; i < rFiles.length; i++) {
+          final Path path = rFiles[i].getPath();
+          sources[i] = new RFileSource(getFileSystem().open(path),
+              getFileSystem().getFileStatus(path).getLen(), rFiles[i].getFence());
         }
       } else {
         for (int i = 0; i < sources.length; i++) {
           if (!(sources[i].getInputStream() instanceof FSDataInputStream)) {
             sources[i] = new RFileSource(new FSDataInputStream(sources[i].getInputStream()),
-                sources[i].getLength());
+                sources[i].getLength(), rFiles[i].getFence());
           }
         }
       }
@@ -128,6 +133,13 @@
   }
 
   @Override
+  public ScannerFSOptions from(FencedPath... files) {
+    Objects.requireNonNull(files);
+    opts.in = new InputArgs(files);
+    return this;
+  }
+
+  @Override
   public ScannerOptions withTableProperties(Iterable<Entry<String,String>> tableConfig) {
     Objects.requireNonNull(tableConfig);
     this.opts.tableConfig = new HashMap<>();
@@ -150,4 +162,5 @@
     this.opts.bounds = range;
     return this;
   }
+
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileSource.java b/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileSource.java
index cb5d0bb..a77e2a4 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileSource.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileSource.java
@@ -19,6 +19,9 @@
 package org.apache.accumulo.core.client.rfile;
 
 import java.io.InputStream;
+import java.util.Objects;
+
+import org.apache.accumulo.core.data.Range;
 
 /**
  * RFile metadata is stored at the end of the file. Inorder to read an RFile, its length must be
@@ -29,10 +32,19 @@
 public class RFileSource {
   private final InputStream in;
   private final long len;
+  private final Range range;
 
   public RFileSource(InputStream in, long len) {
-    this.in = in;
+    this(in, len, new Range());
+  }
+
+  /**
+   * @since 3.1.0
+   */
+  public RFileSource(InputStream in, long len, Range range) {
+    this.in = Objects.requireNonNull(in);
     this.len = len;
+    this.range = Objects.requireNonNull(range);
   }
 
   public InputStream getInputStream() {
@@ -42,4 +54,13 @@
   public long getLength() {
     return len;
   }
+
+  /**
+   * @return The range of the RFileSource
+   *
+   * @since 3.1.0
+   */
+  public Range getRange() {
+    return range;
+  }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileWriterBuilder.java b/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileWriterBuilder.java
index be1850c..b1d7957 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileWriterBuilder.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileWriterBuilder.java
@@ -38,6 +38,7 @@
 import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.crypto.CryptoFactoryLoader;
 import org.apache.accumulo.core.file.FileOperations;
+import org.apache.accumulo.core.metadata.UnreferencedTabletFile;
 import org.apache.accumulo.core.metadata.ValidationUtil;
 import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
 import org.apache.accumulo.core.spi.crypto.CryptoEnvironment;
@@ -119,7 +120,8 @@
           visCacheSize);
     } else {
       return new RFileWriter(fileops.newWriterBuilder()
-          .forFile(out.path.toString(), out.getFileSystem(), out.getConf(), cs)
+          .forFile(UnreferencedTabletFile.of(out.getFileSystem(), out.path), out.getFileSystem(),
+              out.getConf(), cs)
           .withTableConfiguration(acuconf).withStartDisabled().build(), visCacheSize);
     }
   }
diff --git a/core/src/main/java/org/apache/accumulo/core/client/sample/AbstractHashSampler.java b/core/src/main/java/org/apache/accumulo/core/client/sample/AbstractHashSampler.java
index 072f6ca..59895e3 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/sample/AbstractHashSampler.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/sample/AbstractHashSampler.java
@@ -23,6 +23,7 @@
 
 import java.io.DataOutput;
 import java.io.IOException;
+import java.io.UncheckedIOException;
 import java.util.Map;
 import java.util.Set;
 
@@ -85,17 +86,6 @@
   }
 
   /**
-   * Subclasses with options should override this method and return true if the option is valid for
-   * the subclass or if {@code super.isValidOption(opt)} returns true.
-   *
-   * @deprecated since 2.1.0, replaced by {@link #validateOptions(Map)}
-   */
-  @Deprecated(since = "2.1.0")
-  protected boolean isValidOption(String option) {
-    return REQUIRED_SAMPLER_OPTIONS.contains(option);
-  }
-
-  /**
    * Subclasses with options should override this method and call {@code super.init(config)}.
    */
   @SuppressFBWarnings(value = "UNSAFE_HASH_EQUALS",
@@ -142,7 +132,7 @@
     try {
       hash(new DataoutputHasher(hasher), k);
     } catch (IOException e) {
-      throw new RuntimeException(e);
+      throw new UncheckedIOException(e);
     }
     return hasher.hash().asInt() % modulus == 0;
   }
diff --git a/core/src/main/java/org/apache/accumulo/core/client/security/tokens/AuthenticationToken.java b/core/src/main/java/org/apache/accumulo/core/client/security/tokens/AuthenticationToken.java
index 6e53023..0652a3b 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/security/tokens/AuthenticationToken.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/security/tokens/AuthenticationToken.java
@@ -23,6 +23,7 @@
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
+import java.io.UncheckedIOException;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashMap;
@@ -115,7 +116,7 @@
         token.write(out);
         return baos.toByteArray();
       } catch (IOException e) {
-        throw new RuntimeException("Bug found in serialization code", e);
+        throw new UncheckedIOException("Bug found in serialization code", e);
       }
     }
   }
diff --git a/core/src/main/java/org/apache/accumulo/core/client/security/tokens/KerberosToken.java b/core/src/main/java/org/apache/accumulo/core/client/security/tokens/KerberosToken.java
index 50f44c7..a28cc06 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/security/tokens/KerberosToken.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/security/tokens/KerberosToken.java
@@ -25,6 +25,7 @@
 import java.io.DataOutput;
 import java.io.File;
 import java.io.IOException;
+import java.io.UncheckedIOException;
 import java.util.Collections;
 import java.util.Set;
 
@@ -107,8 +108,10 @@
       clone.principal = principal;
       clone.keytab = keytab == null ? keytab : keytab.getCanonicalFile();
       return clone;
-    } catch (CloneNotSupportedException | IOException e) {
-      throw new RuntimeException(e);
+    } catch (CloneNotSupportedException e) {
+      throw new IllegalStateException(e);
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
     }
   }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/client/security/tokens/PasswordToken.java b/core/src/main/java/org/apache/accumulo/core/client/security/tokens/PasswordToken.java
index 8390f0f..30e570c 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/security/tokens/PasswordToken.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/security/tokens/PasswordToken.java
@@ -101,8 +101,8 @@
       AtomicBoolean calledFirstReadInt = new AtomicBoolean(false);
       DataInput wrapped = (DataInput) Proxy.newProxyInstance(DataInput.class.getClassLoader(),
           arg0.getClass().getInterfaces(), (obj, method, args) -> {
-            // wrap the original DataInput in order to return the integer that was read
-            // and then not used, because it didn't match -2
+            // wrap the original DataInput in order to simulate replacing the integer that was
+            // previously read and then not used back into the input, after it didn't match -2
             if (!calledFirstReadInt.get() && method.getName().equals("readInt")) {
               calledFirstReadInt.set(true);
               return version;
@@ -160,7 +160,7 @@
       clone.password = Arrays.copyOf(password, password.length);
       return clone;
     } catch (CloneNotSupportedException e) {
-      throw new RuntimeException(e);
+      throw new IllegalStateException(e);
     }
   }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/client/summary/summarizers/AuthorizationSummarizer.java b/core/src/main/java/org/apache/accumulo/core/client/summary/summarizers/AuthorizationSummarizer.java
index 5660f66..77a9244 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/summary/summarizers/AuthorizationSummarizer.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/summary/summarizers/AuthorizationSummarizer.java
@@ -24,14 +24,13 @@
 import java.util.Set;
 import java.util.function.Consumer;
 
+import org.apache.accumulo.access.AccessExpression;
 import org.apache.accumulo.core.client.admin.TableOperations;
 import org.apache.accumulo.core.client.summary.CountingSummarizer;
 import org.apache.accumulo.core.data.ArrayByteSequence;
 import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.security.ColumnVisibility;
-import org.apache.accumulo.core.security.ColumnVisibility.Node;
 
 /**
  * Counts unique authorizations in column visibility labels. Leverages super class to defend against
@@ -82,7 +81,10 @@
       if (vis.length() > 0) {
         Set<ByteSequence> auths = cache.get(vis);
         if (auths == null) {
-          auths = findAuths(vis);
+          auths = new HashSet<>();
+          for (String auth : AccessExpression.of(vis.toArray()).getAuthorizations().asSet()) {
+            auths.add(new ArrayByteSequence(auth));
+          }
           cache.put(new ArrayByteSequence(vis), auths);
         }
 
@@ -91,33 +93,5 @@
         }
       }
     }
-
-    private Set<ByteSequence> findAuths(ByteSequence vis) {
-      HashSet<ByteSequence> auths = new HashSet<>();
-      byte[] expression = vis.toArray();
-      Node root = new ColumnVisibility(expression).getParseTree();
-
-      findAuths(root, expression, auths);
-
-      return auths;
-    }
-
-    private void findAuths(Node node, byte[] expression, HashSet<ByteSequence> auths) {
-      switch (node.getType()) {
-        case AND:
-        case OR:
-          for (Node child : node.getChildren()) {
-            findAuths(child, expression, auths);
-          }
-          break;
-        case TERM:
-          auths.add(node.getTerm(expression));
-          break;
-        case EMPTY:
-          break;
-        default:
-          throw new IllegalArgumentException("Unknown node type " + node.getType());
-      }
-    }
   }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ActiveCompactionImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ActiveCompactionImpl.java
index b7889c3..ca36f3d 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ActiveCompactionImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ActiveCompactionImpl.java
@@ -30,7 +30,8 @@
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.dataImpl.TabletIdImpl;
 import org.apache.accumulo.core.dataImpl.thrift.IterInfo;
-import org.apache.accumulo.core.util.HostAndPort;
+
+import com.google.common.net.HostAndPort;
 
 /**
  * @since 1.6.0
@@ -102,6 +103,11 @@
   }
 
   @Override
+  public long getPausedCount() {
+    return tac.getTimesPaused();
+  }
+
+  @Override
   public List<IteratorSetting> getIterators() {
     ArrayList<IteratorSetting> ret = new ArrayList<>();
 
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ActiveScanImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ActiveScanImpl.java
index ba9160f..d125fd2 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ActiveScanImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ActiveScanImpl.java
@@ -57,7 +57,7 @@
   private Authorizations authorizations;
 
   ActiveScanImpl(ClientContext context,
-      org.apache.accumulo.core.tabletserver.thrift.ActiveScan activeScan)
+      org.apache.accumulo.core.tabletscan.thrift.ActiveScan activeScan)
       throws TableNotFoundException {
     this.scanId = activeScan.scanId;
     this.client = activeScan.client;
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientConfConverter.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientConfConverter.java
index 3929c3e..2ccc440 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientConfConverter.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientConfConverter.java
@@ -19,8 +19,6 @@
 package org.apache.accumulo.core.clientImpl;
 
 import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
 import java.util.Map;
 import java.util.Properties;
 import java.util.function.Predicate;
@@ -38,121 +36,74 @@
 public class ClientConfConverter {
 
   private static final Logger log = LoggerFactory.getLogger(ClientConfConverter.class);
-  private static Map<String,String> confProps = new HashMap<>();
-  private static Map<String,String> propsConf = new HashMap<>();
-
-  @SuppressWarnings("deprecation")
-  private static void init() {
-    propsConf.put(ClientProperty.INSTANCE_ZOOKEEPERS.getKey(),
-        org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.INSTANCE_ZK_HOST
-            .getKey());
-    propsConf.put(ClientProperty.INSTANCE_ZOOKEEPERS_TIMEOUT.getKey(),
-        org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.INSTANCE_ZK_TIMEOUT
-            .getKey());
-    propsConf.put(ClientProperty.SSL_ENABLED.getKey(),
-        org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.INSTANCE_RPC_SSL_ENABLED
-            .getKey());
-    propsConf.put(ClientProperty.SSL_KEYSTORE_PATH.getKey(),
-        org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.RPC_SSL_KEYSTORE_PATH
-            .getKey());
-    propsConf.put(ClientProperty.SSL_KEYSTORE_TYPE.getKey(),
-        org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.RPC_SSL_KEYSTORE_TYPE
-            .getKey());
-    propsConf.put(ClientProperty.SSL_KEYSTORE_PASSWORD.getKey(),
-        org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.RPC_SSL_KEYSTORE_PASSWORD
-            .getKey());
-    propsConf.put(ClientProperty.SSL_TRUSTSTORE_PATH.getKey(),
-        org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.RPC_SSL_TRUSTSTORE_PATH
-            .getKey());
-    propsConf.put(ClientProperty.SSL_TRUSTSTORE_TYPE.getKey(),
-        org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.RPC_SSL_TRUSTSTORE_TYPE
-            .getKey());
-    propsConf.put(ClientProperty.SSL_TRUSTSTORE_PASSWORD.getKey(),
-        org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.RPC_SSL_TRUSTSTORE_PASSWORD
-            .getKey());
-    propsConf.put(ClientProperty.SSL_USE_JSSE.getKey(),
-        org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.RPC_USE_JSSE.getKey());
-    propsConf.put(ClientProperty.SASL_ENABLED.getKey(),
-        org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.INSTANCE_RPC_SASL_ENABLED
-            .getKey());
-    propsConf.put(ClientProperty.SASL_QOP.getKey(),
-        org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.RPC_SASL_QOP.getKey());
-    propsConf.put(ClientProperty.SASL_KERBEROS_SERVER_PRIMARY.getKey(),
-        org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.KERBEROS_SERVER_PRIMARY
-            .getKey());
-
-    for (Map.Entry<String,String> entry : propsConf.entrySet()) {
-      confProps.put(entry.getValue(), entry.getKey());
-    }
-  }
+  private static final Map<String,String> accumuloConfToClientProps = new HashMap<>();
+  private static final Map<String,String> clientPropsToAccumuloConf = new HashMap<>();
 
   static {
-    init();
-  }
+    // mapping of ClientProperty equivalents in AccumuloConfiguration
+    Map<ClientProperty,Property> conversions = new HashMap<>();
+    conversions.put(ClientProperty.INSTANCE_ZOOKEEPERS, Property.INSTANCE_ZK_HOST);
+    conversions.put(ClientProperty.INSTANCE_ZOOKEEPERS_TIMEOUT, Property.INSTANCE_ZK_TIMEOUT);
 
-  @SuppressWarnings("deprecation")
-  public static org.apache.accumulo.core.client.ClientConfiguration
-      toClientConf(Properties properties) {
-    org.apache.accumulo.core.client.ClientConfiguration config =
-        org.apache.accumulo.core.client.ClientConfiguration.create();
-    for (Object keyObj : properties.keySet()) {
-      String propKey = (String) keyObj;
-      String val = properties.getProperty(propKey);
-      String confKey = propsConf.get(propKey);
-      if (confKey == null) {
-        config.setProperty(propKey, val);
-      } else {
-        config.setProperty(confKey, val);
-      }
-      if (propKey.equals(ClientProperty.SSL_KEYSTORE_PATH.getKey())) {
-        config.setProperty(
-            org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.INSTANCE_RPC_SSL_CLIENT_AUTH,
-            "true");
-      }
-    }
-    return config;
-  }
+    conversions.put(ClientProperty.SASL_ENABLED, Property.INSTANCE_RPC_SASL_ENABLED);
+    conversions.put(ClientProperty.SASL_QOP, Property.RPC_SASL_QOP);
 
-  @SuppressWarnings("deprecation")
-  public static Properties
-      toProperties(org.apache.accumulo.core.client.ClientConfiguration clientConf) {
-    Properties props = new Properties();
-    Iterator<String> clientConfIter = clientConf.getKeys();
-    while (clientConfIter.hasNext()) {
-      String confKey = clientConfIter.next();
-      String val = clientConf.getString(confKey);
-      String propKey = confProps.get(confKey);
-      if (propKey == null) {
-        if (!confKey.equals(
-            org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.INSTANCE_RPC_SSL_CLIENT_AUTH
-                .getKey())) {
-          props.setProperty(confKey, val);
-        }
-      } else {
-        props.setProperty(propKey, val);
-      }
+    conversions.put(ClientProperty.SSL_ENABLED, Property.INSTANCE_RPC_SSL_ENABLED);
+    conversions.put(ClientProperty.SSL_KEYSTORE_PASSWORD, Property.RPC_SSL_KEYSTORE_PASSWORD);
+    conversions.put(ClientProperty.SSL_KEYSTORE_PATH, Property.RPC_SSL_KEYSTORE_PATH);
+    conversions.put(ClientProperty.SSL_KEYSTORE_TYPE, Property.RPC_SSL_KEYSTORE_TYPE);
+    conversions.put(ClientProperty.SSL_TRUSTSTORE_PASSWORD, Property.RPC_SSL_TRUSTSTORE_PASSWORD);
+    conversions.put(ClientProperty.SSL_TRUSTSTORE_PATH, Property.RPC_SSL_TRUSTSTORE_PATH);
+    conversions.put(ClientProperty.SSL_TRUSTSTORE_TYPE, Property.RPC_SSL_TRUSTSTORE_TYPE);
+    conversions.put(ClientProperty.SSL_USE_JSSE, Property.RPC_USE_JSSE);
+
+    for (Map.Entry<ClientProperty,Property> entry : conversions.entrySet()) {
+      accumuloConfToClientProps.put(entry.getValue().getKey(), entry.getKey().getKey());
+      clientPropsToAccumuloConf.put(entry.getKey().getKey(), entry.getValue().getKey());
     }
-    return props;
   }
 
   public static Properties toProperties(AccumuloConfiguration config) {
-    return toProperties(toClientConf(config));
+    final var propsExtractedFromConfig = new Properties();
+
+    // Extract kerberos primary from the config
+    final String serverPrincipal = config.get(Property.GENERAL_KERBEROS_PRINCIPAL);
+    if (serverPrincipal != null && !serverPrincipal.isEmpty()) {
+      var krbName = new KerberosName(serverPrincipal);
+      propsExtractedFromConfig.setProperty(ClientProperty.SASL_KERBEROS_SERVER_PRIMARY.getKey(),
+          krbName.getServiceName());
+    }
+
+    // Extract the remaining properties from the config
+    config.stream().filter(e -> accumuloConfToClientProps.keySet().contains(e.getKey()))
+        .forEach(e -> propsExtractedFromConfig.setProperty(e.getKey(), e.getValue()));
+
+    // For all the extracted properties, convert them to their ClientProperty names
+    final var convertedProps = new Properties();
+    propsExtractedFromConfig.forEach((k, v) -> {
+      String confKey = String.valueOf(k);
+      String val = String.valueOf(v);
+      String propKey = accumuloConfToClientProps.get(confKey);
+      convertedProps.setProperty(propKey == null ? confKey : propKey, val);
+    });
+    return convertedProps;
   }
 
   public static AccumuloConfiguration toAccumuloConf(Properties properties) {
-    return toAccumuloConf(toClientConf(properties));
-  }
-
-  /**
-   * A utility method for converting client configuration to a standard configuration object for use
-   * internally.
-   *
-   * @param config the original config
-   * @return the client configuration presented in the form of an {@link AccumuloConfiguration}
-   */
-  @SuppressWarnings("deprecation")
-  public static AccumuloConfiguration
-      toAccumuloConf(final org.apache.accumulo.core.client.ClientConfiguration config) {
+    final var convertedProps = new Properties();
+    for (String propKey : properties.stringPropertyNames()) {
+      String val = properties.getProperty(propKey);
+      String confKey = clientPropsToAccumuloConf.get(propKey);
+      if (propKey.equals(ClientProperty.SASL_KERBEROS_SERVER_PRIMARY.getKey())) {
+        confKey = Property.GENERAL_KERBEROS_PRINCIPAL.getKey();
+        // Avoid providing a realm since we don't know what it is...
+        val += "/_HOST@" + SaslConnectionParams.getDefaultRealm();
+      }
+      convertedProps.setProperty(confKey == null ? propKey : confKey, val);
+      if (propKey.equals(ClientProperty.SSL_KEYSTORE_PATH.getKey())) {
+        convertedProps.setProperty(Property.INSTANCE_RPC_SSL_CLIENT_AUTH.getKey(), "true");
+      }
+    }
 
     final AccumuloConfiguration defaults = DefaultConfiguration.getInstance();
 
@@ -160,7 +111,7 @@
 
       @Override
       public boolean isPropertySet(Property prop) {
-        return config.containsKey(prop.getKey());
+        return convertedProps.containsKey(prop.getKey());
       }
 
       @Override
@@ -181,51 +132,15 @@
             }
           }
         }
-
-        if (config.containsKey(key)) {
-          return config.getString(key);
-        } else {
-          // Reconstitute the server kerberos property from the client config
-          if (property == Property.GENERAL_KERBEROS_PRINCIPAL) {
-            if (config.containsKey(
-                org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.KERBEROS_SERVER_PRIMARY
-                    .getKey())) {
-              // Avoid providing a realm since we don't know what it is...
-              return config.getString(
-                  org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.KERBEROS_SERVER_PRIMARY
-                      .getKey())
-                  + "/_HOST@" + SaslConnectionParams.getDefaultRealm();
-            }
-          }
-          return defaults.get(property);
-        }
+        return convertedProps.getProperty(key, defaults.get(property));
       }
 
       @Override
       public void getProperties(Map<String,String> props, Predicate<String> filter) {
         defaults.getProperties(props, filter);
-
-        Iterator<String> keyIter = config.getKeys();
-        while (keyIter.hasNext()) {
-          String key = keyIter.next();
+        for (String key : convertedProps.stringPropertyNames()) {
           if (filter.test(key)) {
-            props.put(key, config.getString(key));
-          }
-        }
-
-        // Two client props that don't exist on the server config. Client doesn't need to know about
-        // the Kerberos instance from the principle, but servers do
-        // Automatically reconstruct the server property when converting a client config.
-        if (props.containsKey(
-            org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.KERBEROS_SERVER_PRIMARY
-                .getKey())) {
-          final String serverPrimary = props.remove(
-              org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.KERBEROS_SERVER_PRIMARY
-                  .getKey());
-          if (filter.test(Property.GENERAL_KERBEROS_PRINCIPAL.getKey())) {
-            // Use the _HOST expansion. It should be unnecessary in "client land".
-            props.put(Property.GENERAL_KERBEROS_PRINCIPAL.getKey(),
-                serverPrimary + "/_HOST@" + SaslConnectionParams.getDefaultRealm());
+            props.put(key, convertedProps.getProperty(key));
           }
         }
 
@@ -247,8 +162,8 @@
       }
 
       private org.apache.hadoop.conf.Configuration getHadoopConfiguration() {
-        String credProviderPaths =
-            config.getString(Property.GENERAL_SECURITY_CREDENTIAL_PROVIDER_PATHS.getKey());
+        String credProviderPaths = convertedProps
+            .getProperty(Property.GENERAL_SECURITY_CREDENTIAL_PROVIDER_PATHS.getKey());
         if (credProviderPaths != null && !credProviderPaths.isEmpty()) {
           org.apache.hadoop.conf.Configuration hConf = new org.apache.hadoop.conf.Configuration();
           HadoopCredentialProvider.setPath(hConf, credProviderPaths);
@@ -259,40 +174,7 @@
 
         return null;
       }
+
     };
   }
-
-  @SuppressWarnings("deprecation")
-  public static org.apache.accumulo.core.client.ClientConfiguration
-      toClientConf(AccumuloConfiguration conf) {
-    org.apache.accumulo.core.client.ClientConfiguration clientConf =
-        org.apache.accumulo.core.client.ClientConfiguration.create();
-
-    // Servers will only have the full principal in their configuration -- parse the
-    // primary and realm from it.
-    final String serverPrincipal = conf.get(Property.GENERAL_KERBEROS_PRINCIPAL);
-
-    final KerberosName krbName;
-    if (serverPrincipal != null && !serverPrincipal.isEmpty()) {
-      krbName = new KerberosName(serverPrincipal);
-      clientConf.setProperty(
-          org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.KERBEROS_SERVER_PRIMARY,
-          krbName.getServiceName());
-    }
-
-    HashSet<String> clientKeys = new HashSet<>();
-    for (org.apache.accumulo.core.client.ClientConfiguration.ClientProperty prop : org.apache.accumulo.core.client.ClientConfiguration.ClientProperty
-        .values()) {
-      clientKeys.add(prop.getKey());
-    }
-
-    String key;
-    for (Map.Entry<String,String> entry : conf) {
-      key = entry.getKey();
-      if (clientKeys.contains(key)) {
-        clientConf.setProperty(key, entry.getValue());
-      }
-    }
-    return clientConf;
-  }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
index 75b6394..eb77916 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
@@ -22,7 +22,6 @@
 import static com.google.common.base.Suppliers.memoize;
 import static com.google.common.base.Suppliers.memoizeWithExpiration;
 import static java.nio.charset.StandardCharsets.UTF_8;
-import static java.util.Objects.requireNonNull;
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static java.util.concurrent.TimeUnit.SECONDS;
 import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.LOCATION;
@@ -37,6 +36,7 @@
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
+import java.util.Optional;
 import java.util.Properties;
 import java.util.UUID;
 import java.util.concurrent.Callable;
@@ -76,12 +76,14 @@
 import org.apache.accumulo.core.data.KeyValue;
 import org.apache.accumulo.core.data.NamespaceId;
 import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock;
 import org.apache.accumulo.core.fate.zookeeper.ZooCache;
 import org.apache.accumulo.core.fate.zookeeper.ZooCache.ZcStat;
 import org.apache.accumulo.core.fate.zookeeper.ZooCacheFactory;
 import org.apache.accumulo.core.fate.zookeeper.ZooReader;
 import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
+import org.apache.accumulo.core.lock.ServiceLock;
+import org.apache.accumulo.core.lock.ServiceLockData;
+import org.apache.accumulo.core.lock.ServiceLockData.ThriftService;
 import org.apache.accumulo.core.manager.state.tables.TableState;
 import org.apache.accumulo.core.metadata.RootTable;
 import org.apache.accumulo.core.metadata.schema.Ample;
@@ -148,8 +150,6 @@
   private final TableOperationsImpl tableops;
   private final NamespaceOperations namespaceops;
   private InstanceOperations instanceops = null;
-  @SuppressWarnings("deprecation")
-  private org.apache.accumulo.core.client.admin.ReplicationOperations replicationops = null;
   private final SingletonReservation singletonReservation;
   private final ThreadPools clientThreadPools;
   private ThreadPoolExecutor cleanupThreadPool;
@@ -406,11 +406,10 @@
       try {
         final var zLockPath = ServiceLock.path(root + "/" + addr);
         ZcStat stat = new ZcStat();
-        byte[] lockData = ServiceLock.getLockData(getZooCache(), zLockPath, stat);
-        if (lockData != null) {
-          String[] fields = new String(lockData, UTF_8).split(",", 2);
-          UUID uuid = UUID.fromString(fields[0]);
-          String group = fields[1];
+        Optional<ServiceLockData> sld = ServiceLock.getLockData(getZooCache(), zLockPath, stat);
+        if (sld.isPresent()) {
+          UUID uuid = sld.orElseThrow().getServerUUID(ThriftService.TABLET_SCAN);
+          String group = sld.orElseThrow().getGroup(ThriftService.TABLET_SCAN);
           liveScanServers.put(addr, new Pair<>(uuid, group));
         }
       } catch (IllegalArgumentException e) {
@@ -510,35 +509,34 @@
    */
   public List<String> getManagerLocations() {
     ensureOpen();
-    return getManagerLocations(zooCache, getInstanceID().canonical());
-  }
-
-  // available only for sharing code with old ZooKeeperInstance
-  public static List<String> getManagerLocations(ZooCache zooCache, String instanceId) {
     var zLockManagerPath =
-        ServiceLock.path(Constants.ZROOT + "/" + instanceId + Constants.ZMANAGER_LOCK);
+        ServiceLock.path(Constants.ZROOT + "/" + getInstanceID() + Constants.ZMANAGER_LOCK);
 
     OpTimer timer = null;
 
     if (log.isTraceEnabled()) {
-      log.trace("tid={} Looking up manager location in zookeeper.", Thread.currentThread().getId());
+      log.trace("tid={} Looking up manager location in zookeeper at {}.",
+          Thread.currentThread().getId(), zLockManagerPath);
       timer = new OpTimer().start();
     }
 
-    byte[] loc = zooCache.getLockData(zLockManagerPath);
+    Optional<ServiceLockData> sld = zooCache.getLockData(zLockManagerPath);
+    String location = null;
+    if (sld.isPresent()) {
+      location = sld.orElseThrow().getAddressString(ThriftService.MANAGER);
+    }
 
     if (timer != null) {
       timer.stop();
       log.trace("tid={} Found manager at {} in {}", Thread.currentThread().getId(),
-          (loc == null ? "null" : new String(loc, UTF_8)),
-          String.format("%.3f secs", timer.scale(SECONDS)));
+          (location == null ? "null" : location), String.format("%.3f secs", timer.scale(SECONDS)));
     }
 
-    if (loc == null) {
+    if (location == null) {
       return Collections.emptyList();
     }
 
-    return Collections.singletonList(new String(loc, UTF_8));
+    return Collections.singletonList(location);
   }
 
   /**
@@ -549,37 +547,27 @@
   public InstanceId getInstanceID() {
     ensureOpen();
     if (instanceId == null) {
+      // lookup by name
       final String instanceName = info.getInstanceName();
-      instanceId = getInstanceID(zooCache, instanceName);
-      verifyInstanceId(zooCache, instanceId.canonical(), instanceName);
+      String instanceNamePath = Constants.ZROOT + Constants.ZINSTANCES + "/" + instanceName;
+      byte[] data = zooCache.get(instanceNamePath);
+      if (data == null) {
+        throw new RuntimeException(
+            "Instance name " + instanceName + " does not exist in zookeeper. "
+                + "Run \"accumulo org.apache.accumulo.server.util.ListInstances\" to see a list.");
+      }
+      String instanceIdString = new String(data, UTF_8);
+      // verify that the instanceId found via the instanceName actually exists as an instance
+      if (zooCache.get(Constants.ZROOT + "/" + instanceIdString) == null) {
+        throw new RuntimeException("Instance id " + instanceIdString
+            + (instanceName == null ? "" : " pointed to by the name " + instanceName)
+            + " does not exist in zookeeper");
+      }
+      instanceId = InstanceId.of(instanceIdString);
     }
     return instanceId;
   }
 
-  // available only for sharing code with old ZooKeeperInstance
-  public static InstanceId getInstanceID(ZooCache zooCache, String instanceName) {
-    requireNonNull(zooCache, "zooCache cannot be null");
-    requireNonNull(instanceName, "instanceName cannot be null");
-    String instanceNamePath = Constants.ZROOT + Constants.ZINSTANCES + "/" + instanceName;
-    byte[] data = zooCache.get(instanceNamePath);
-    if (data == null) {
-      throw new RuntimeException("Instance name " + instanceName + " does not exist in zookeeper. "
-          + "Run \"accumulo org.apache.accumulo.server.util.ListInstances\" to see a list.");
-    }
-    return InstanceId.of(new String(data, UTF_8));
-  }
-
-  // available only for sharing code with old ZooKeeperInstance
-  public static void verifyInstanceId(ZooCache zooCache, String instanceId, String instanceName) {
-    requireNonNull(zooCache, "zooCache cannot be null");
-    requireNonNull(instanceId, "instanceId cannot be null");
-    if (zooCache.get(Constants.ZROOT + "/" + instanceId) == null) {
-      throw new RuntimeException("Instance id " + instanceId
-          + (instanceName == null ? "" : " pointed to by the name " + instanceName)
-          + " does not exist in zookeeper");
-    }
-  }
-
   public String getZooKeeperRoot() {
     ensureOpen();
     return ZooUtil.getRoot(getInstanceID());
@@ -849,18 +837,6 @@
   }
 
   @Override
-  @Deprecated
-  public synchronized org.apache.accumulo.core.client.admin.ReplicationOperations
-      replicationOperations() {
-    ensureOpen();
-    if (replicationops == null) {
-      replicationops = new ReplicationOperationsImpl(this);
-    }
-
-    return replicationops;
-  }
-
-  @Override
   public Properties properties() {
     ensureOpen();
     Properties result = new Properties();
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientServiceEnvironmentImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientServiceEnvironmentImpl.java
index ad7827c..699fa49 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientServiceEnvironmentImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientServiceEnvironmentImpl.java
@@ -18,8 +18,6 @@
  */
 package org.apache.accumulo.core.clientImpl;
 
-import java.io.IOException;
-
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.TableNotFoundException;
@@ -62,15 +60,14 @@
   }
 
   @Override
-  public <T> T instantiate(String className, Class<T> base)
-      throws ReflectiveOperationException, IOException {
+  public <T> T instantiate(String className, Class<T> base) throws ReflectiveOperationException {
     return ClientServiceEnvironmentImpl.class.getClassLoader().loadClass(className).asSubclass(base)
         .getDeclaredConstructor().newInstance();
   }
 
   @Override
   public <T> T instantiate(TableId tableId, String className, Class<T> base)
-      throws ReflectiveOperationException, IOException {
+      throws ReflectiveOperationException {
     return instantiate(className, base);
   }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/CompactionStrategyConfigUtil.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/CompactionStrategyConfigUtil.java
deleted file mode 100644
index f09adc2..0000000
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/CompactionStrategyConfigUtil.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.clientImpl;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.util.Map;
-
-import org.apache.accumulo.core.client.admin.CompactionConfig;
-import org.apache.accumulo.core.client.admin.CompactionStrategyConfig;
-
-@SuppressWarnings("removal")
-public class CompactionStrategyConfigUtil {
-
-  public static final CompactionStrategyConfig DEFAULT_STRATEGY = new CompactionStrategyConfig("") {
-    @Override
-    public CompactionStrategyConfig setOptions(Map<String,String> opts) {
-      throw new UnsupportedOperationException();
-    }
-  };
-
-  private static final int MAGIC = 0xcc5e6024;
-
-  public static void encode(DataOutput dout, CompactionConfig cc) {
-    var cs = cc.getCompactionStrategy();
-    UserCompactionUtils.encode(dout, MAGIC, 1, cs.getClassName(), cs.getOptions());
-  }
-
-  public static void decode(CompactionConfig cc, DataInput din) {
-    var pcd = UserCompactionUtils.decode(din, MAGIC, 1);
-    var csc = new CompactionStrategyConfig(pcd.className).setOptions(pcd.opts);
-
-    if (!isDefault(csc)) {
-      cc.setCompactionStrategy(csc);
-    }
-  }
-
-  public static boolean isDefault(CompactionStrategyConfig compactionStrategy) {
-    return compactionStrategy.equals(DEFAULT_STRATEGY);
-
-  }
-
-  public static boolean isDefault(CompactionConfig compactionConfig) {
-    return isDefault(compactionConfig.getCompactionStrategy());
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ConditionalWriterImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ConditionalWriterImpl.java
index cb76751..654135a 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ConditionalWriterImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ConditionalWriterImpl.java
@@ -18,10 +18,10 @@
  */
 package org.apache.accumulo.core.clientImpl;
 
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 import static java.nio.charset.StandardCharsets.UTF_8;
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static java.util.concurrent.TimeUnit.SECONDS;
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
 
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
@@ -42,6 +42,8 @@
 import java.util.concurrent.ScheduledThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.accumulo.access.AccessEvaluator;
+import org.apache.accumulo.access.InvalidAccessExpressionException;
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -50,6 +52,7 @@
 import org.apache.accumulo.core.client.Durability;
 import org.apache.accumulo.core.client.TimedOutException;
 import org.apache.accumulo.core.clientImpl.TabletLocator.TabletServerMutations;
+import org.apache.accumulo.core.clientImpl.thrift.TInfo;
 import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.data.Condition;
@@ -63,21 +66,15 @@
 import org.apache.accumulo.core.dataImpl.thrift.TConditionalSession;
 import org.apache.accumulo.core.dataImpl.thrift.TKeyExtent;
 import org.apache.accumulo.core.dataImpl.thrift.TMutation;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock;
 import org.apache.accumulo.core.fate.zookeeper.ZooUtil.LockID;
+import org.apache.accumulo.core.lock.ServiceLock;
 import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.ColumnVisibility;
-import org.apache.accumulo.core.security.VisibilityEvaluator;
-import org.apache.accumulo.core.security.VisibilityParseException;
+import org.apache.accumulo.core.tabletingest.thrift.TabletIngestClientService;
 import org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException;
-import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
 import org.apache.accumulo.core.trace.TraceUtil;
-import org.apache.accumulo.core.trace.thrift.TInfo;
-import org.apache.accumulo.core.util.BadArgumentException;
 import org.apache.accumulo.core.util.ByteBufferUtil;
-import org.apache.accumulo.core.util.HostAndPort;
 import org.apache.accumulo.core.util.threads.ThreadPools;
 import org.apache.accumulo.core.util.threads.Threads;
 import org.apache.commons.collections4.map.LRUMap;
@@ -91,6 +88,7 @@
 import org.slf4j.LoggerFactory;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.net.HostAndPort;
 
 public class ConditionalWriterImpl implements ConditionalWriter {
 
@@ -98,14 +96,14 @@
 
   private static final int MAX_SLEEP = 30000;
 
-  private Authorizations auths;
-  private VisibilityEvaluator ve;
-  private Map<Text,Boolean> cache = Collections.synchronizedMap(new LRUMap<>(1000));
+  private final Authorizations auths;
+  private final AccessEvaluator accessEvaluator;
+  private final Map<Text,Boolean> cache = Collections.synchronizedMap(new LRUMap<>(1000));
   private final ClientContext context;
-  private TabletLocator locator;
+  private final TabletLocator locator;
   private final TableId tableId;
   private final String tableName;
-  private long timeout;
+  private final long timeout;
   private final Durability durability;
   private final String classLoaderContext;
   private final ConditionalWriterConfig config;
@@ -154,7 +152,7 @@
         count--;
         return result;
       } catch (InterruptedException e) {
-        throw new RuntimeException(e);
+        throw new IllegalStateException(e);
       }
     }
 
@@ -233,7 +231,7 @@
 
     @Override
     public void run() {
-      TabletClientService.Iface client = null;
+      TabletIngestClientService.Iface client = null;
 
       for (SessionID sid : sessions) {
         if (!sid.isActive()) {
@@ -377,7 +375,7 @@
     this.config = config;
     this.context = context;
     this.auths = config.getAuthorizations();
-    this.ve = new VisibilityEvaluator(config.getAuthorizations());
+    this.accessEvaluator = AccessEvaluator.of(config.getAuthorizations().toAccessAuthorizations());
     this.threadPool = context.threadPools().createScheduledExecutorService(
         config.getMaxWriteThreads(), this.getClass().getSimpleName());
     this.locator = new SyncingTabletLocator(context, tableId);
@@ -486,7 +484,7 @@
 
   private HashMap<HostAndPort,SessionID> cachedSessionIDs = new HashMap<>();
 
-  private SessionID reserveSessionID(HostAndPort location, TabletClientService.Iface client,
+  private SessionID reserveSessionID(HostAndPort location, TabletIngestClientService.Iface client,
       TInfo tinfo) throws ThriftSecurityException, TException {
     // avoid cost of repeatedly making RPC to create sessions, reuse sessions
     synchronized (cachedSessionIDs) {
@@ -555,18 +553,19 @@
     return activeSessions;
   }
 
-  private TabletClientService.Iface getClient(HostAndPort location) throws TTransportException {
-    TabletClientService.Iface client;
+  private TabletIngestClientService.Iface getClient(HostAndPort location)
+      throws TTransportException {
+    TabletIngestClientService.Iface client;
     if (timeout < context.getClientTimeoutInMillis()) {
-      client = ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, location, context, timeout);
+      client = ThriftUtil.getClient(ThriftClientTypes.TABLET_INGEST, location, context, timeout);
     } else {
-      client = ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, location, context);
+      client = ThriftUtil.getClient(ThriftClientTypes.TABLET_INGEST, location, context);
     }
     return client;
   }
 
   private void sendToServer(HostAndPort location, TabletServerMutations<QCMutation> mutations) {
-    TabletClientService.Iface client = null;
+    TabletIngestClientService.Iface client = null;
 
     TInfo tinfo = TraceUtil.traceInfo();
 
@@ -723,7 +722,7 @@
   }
 
   private void invalidateSession(long sessionId, HostAndPort location) throws TException {
-    TabletClientService.Iface client = null;
+    TabletIngestClientService.Iface client = null;
 
     TInfo tinfo = TraceUtil.traceInfo();
 
@@ -810,21 +809,24 @@
   }
 
   private boolean isVisible(ByteSequence cv) {
-    Text testVis = new Text(cv.toArray());
-    if (testVis.getLength() == 0) {
+
+    if (cv.length() == 0) {
       return true;
     }
 
+    byte[] arrayVis = cv.toArray();
+    Text testVis = new Text(arrayVis);
+
     Boolean b = cache.get(testVis);
     if (b != null) {
       return b;
     }
 
     try {
-      boolean bb = ve.evaluate(new ColumnVisibility(testVis));
+      boolean bb = accessEvaluator.canAccess(arrayVis);
       cache.put(new Text(testVis), bb);
       return bb;
-    } catch (VisibilityParseException | BadArgumentException e) {
+    } catch (InvalidAccessExpressionException e) {
       return false;
     }
   }
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ConnectorImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ConnectorImpl.java
deleted file mode 100644
index e55375a..0000000
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ConnectorImpl.java
+++ /dev/null
@@ -1,212 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.clientImpl;
-
-import static java.util.concurrent.TimeUnit.MILLISECONDS;
-
-import java.util.List;
-
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.BatchDeleter;
-import org.apache.accumulo.core.client.BatchScanner;
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.BatchWriterConfig;
-import org.apache.accumulo.core.client.ConditionalWriter;
-import org.apache.accumulo.core.client.ConditionalWriterConfig;
-import org.apache.accumulo.core.client.MultiTableBatchWriter;
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.admin.InstanceOperations;
-import org.apache.accumulo.core.client.admin.NamespaceOperations;
-import org.apache.accumulo.core.client.admin.ReplicationOperations;
-import org.apache.accumulo.core.client.admin.SecurityOperations;
-import org.apache.accumulo.core.client.admin.TableOperations;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode;
-import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.singletons.SingletonManager;
-import org.apache.accumulo.core.singletons.SingletonManager.Mode;
-import org.apache.accumulo.core.trace.TraceUtil;
-
-/**
- * This class now delegates to {@link ClientContext}, except for the methods which were not copied
- * over to that.
- */
-@Deprecated(since = "2.0.0")
-public class ConnectorImpl extends org.apache.accumulo.core.client.Connector {
-
-  private static final String SYSTEM_TOKEN_NAME =
-      "org.apache.accumulo.server.security.SystemCredentials$SystemToken";
-  private final ClientContext context;
-
-  public ConnectorImpl(ClientContext context) throws AccumuloSecurityException, AccumuloException {
-    this.context = context;
-    SingletonManager.setMode(Mode.CONNECTOR);
-    if (context.getCredentials().getToken().isDestroyed()) {
-      throw new AccumuloSecurityException(context.getCredentials().getPrincipal(),
-          SecurityErrorCode.TOKEN_EXPIRED);
-    }
-    // Skip fail fast for system services; string literal for class name, to avoid dependency on
-    // server jar
-    final String tokenClassName = context.getCredentials().getToken().getClass().getName();
-    if (!SYSTEM_TOKEN_NAME.equals(tokenClassName)) {
-      if (!ThriftClientTypes.CLIENT.execute(context,
-          client -> client.authenticate(TraceUtil.traceInfo(), context.rpcCreds()))) {
-        throw new AccumuloSecurityException("Authentication failed, access denied",
-            SecurityErrorCode.BAD_CREDENTIALS);
-      }
-    }
-  }
-
-  public ClientContext getAccumuloClient() {
-    return context;
-  }
-
-  @Override
-  public org.apache.accumulo.core.client.Instance getInstance() {
-    return new org.apache.accumulo.core.client.Instance() {
-      @Override
-      public String getRootTabletLocation() {
-        return context.getRootTabletLocation();
-      }
-
-      @Override
-      public List<String> getMasterLocations() {
-        return context.getManagerLocations();
-      }
-
-      @Override
-      public String getInstanceID() {
-        return context.getInstanceID().canonical();
-      }
-
-      @Override
-      public String getInstanceName() {
-        return context.getInstanceName();
-      }
-
-      @Override
-      public String getZooKeepers() {
-        return context.getZooKeepers();
-      }
-
-      @Override
-      public int getZooKeepersSessionTimeOut() {
-        return context.getZooKeepersSessionTimeOut();
-      }
-
-      @Override
-      public org.apache.accumulo.core.client.Connector getConnector(String principal,
-          AuthenticationToken token) throws AccumuloException, AccumuloSecurityException {
-        return org.apache.accumulo.core.client.Connector.from(context);
-      }
-    };
-  }
-
-  @Override
-  public BatchScanner createBatchScanner(String tableName, Authorizations authorizations,
-      int numQueryThreads) throws TableNotFoundException {
-    return context.createBatchScanner(tableName, authorizations, numQueryThreads);
-  }
-
-  @Override
-  public BatchDeleter createBatchDeleter(String tableName, Authorizations authorizations,
-      int numQueryThreads, long maxMemory, long maxLatency, int maxWriteThreads)
-      throws TableNotFoundException {
-    return context.createBatchDeleter(tableName, authorizations, numQueryThreads,
-        new BatchWriterConfig().setMaxMemory(maxMemory).setMaxLatency(maxLatency, MILLISECONDS)
-            .setMaxWriteThreads(maxWriteThreads));
-  }
-
-  @Override
-  public BatchDeleter createBatchDeleter(String tableName, Authorizations authorizations,
-      int numQueryThreads, BatchWriterConfig config) throws TableNotFoundException {
-    return context.createBatchDeleter(tableName, authorizations, numQueryThreads, config);
-  }
-
-  @Override
-  public BatchWriter createBatchWriter(String tableName, long maxMemory, long maxLatency,
-      int maxWriteThreads) throws TableNotFoundException {
-    return context.createBatchWriter(tableName, new BatchWriterConfig().setMaxMemory(maxMemory)
-        .setMaxLatency(maxLatency, MILLISECONDS).setMaxWriteThreads(maxWriteThreads));
-  }
-
-  @Override
-  public BatchWriter createBatchWriter(String tableName, BatchWriterConfig config)
-      throws TableNotFoundException {
-    return context.createBatchWriter(tableName, config);
-  }
-
-  @Override
-  public MultiTableBatchWriter createMultiTableBatchWriter(long maxMemory, long maxLatency,
-      int maxWriteThreads) {
-    return context.createMultiTableBatchWriter(new BatchWriterConfig().setMaxMemory(maxMemory)
-        .setMaxLatency(maxLatency, MILLISECONDS).setMaxWriteThreads(maxWriteThreads));
-  }
-
-  @Override
-  public MultiTableBatchWriter createMultiTableBatchWriter(BatchWriterConfig config) {
-    return context.createMultiTableBatchWriter(config);
-  }
-
-  @Override
-  public ConditionalWriter createConditionalWriter(String tableName, ConditionalWriterConfig config)
-      throws TableNotFoundException {
-    return context.createConditionalWriter(tableName, config);
-  }
-
-  @Override
-  public Scanner createScanner(String tableName, Authorizations authorizations)
-      throws TableNotFoundException {
-    return context.createScanner(tableName, authorizations);
-  }
-
-  @Override
-  public String whoami() {
-    return context.whoami();
-  }
-
-  @Override
-  public TableOperations tableOperations() {
-    return context.tableOperations();
-  }
-
-  @Override
-  public NamespaceOperations namespaceOperations() {
-    return context.namespaceOperations();
-  }
-
-  @Override
-  public SecurityOperations securityOperations() {
-    return context.securityOperations();
-  }
-
-  @Override
-  public InstanceOperations instanceOperations() {
-    return context.instanceOperations();
-  }
-
-  @Override
-  public ReplicationOperations replicationOperations() {
-    return context.replicationOperations();
-  }
-
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/Credentials.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/Credentials.java
index e32047b..eb7dad6 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/Credentials.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/Credentials.java
@@ -86,14 +86,14 @@
    *
    * @param instanceID Accumulo instance ID
    * @return Thrift credentials
-   * @throws RuntimeException if the authentication token has been destroyed (expired)
+   * @throws IllegalStateException if the authentication token has been destroyed (expired)
    */
   public TCredentials toThrift(InstanceId instanceID) {
     TCredentials tCreds = new TCredentials(getPrincipal(), getToken().getClass().getName(),
         ByteBuffer.wrap(AuthenticationTokenSerializer.serialize(getToken())),
         instanceID.canonical());
     if (getToken().isDestroyed()) {
-      throw new RuntimeException("Token has been destroyed",
+      throw new IllegalStateException("Token has been destroyed",
           new AccumuloSecurityException(getPrincipal(), SecurityErrorCode.TOKEN_EXPIRED));
     }
     return tCreds;
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/DurabilityImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/DurabilityImpl.java
index f59f9f3..dd7ce09 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/DurabilityImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/DurabilityImpl.java
@@ -19,7 +19,7 @@
 package org.apache.accumulo.core.clientImpl;
 
 import org.apache.accumulo.core.client.Durability;
-import org.apache.accumulo.core.tabletserver.thrift.TDurability;
+import org.apache.accumulo.core.tabletingest.thrift.TDurability;
 
 public class DurabilityImpl {
 
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 084d59e..215f7c6 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
@@ -20,15 +20,13 @@
 
 import static com.google.common.base.Preconditions.checkArgument;
 import static java.nio.charset.StandardCharsets.UTF_8;
-import static java.util.concurrent.TimeUnit.MILLISECONDS;
-import static java.util.concurrent.TimeUnit.MINUTES;
-import static java.util.concurrent.TimeUnit.SECONDS;
 import static java.util.stream.Collectors.toList;
 import static org.apache.accumulo.core.rpc.ThriftUtil.createClient;
 import static org.apache.accumulo.core.rpc.ThriftUtil.createTransport;
 import static org.apache.accumulo.core.rpc.ThriftUtil.getClient;
 import static org.apache.accumulo.core.rpc.ThriftUtil.returnClient;
 
+import java.time.Duration;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.ConcurrentModificationException;
@@ -56,11 +54,10 @@
 import org.apache.accumulo.core.data.InstanceId;
 import org.apache.accumulo.core.fate.zookeeper.ZooCache;
 import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
-import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Client;
-import org.apache.accumulo.core.tabletserver.thrift.TabletScanClientService;
+import org.apache.accumulo.core.tabletscan.thrift.TabletScanClientService;
+import org.apache.accumulo.core.tabletserver.thrift.TabletServerClientService.Client;
 import org.apache.accumulo.core.trace.TraceUtil;
 import org.apache.accumulo.core.util.AddressUtil;
-import org.apache.accumulo.core.util.HostAndPort;
 import org.apache.accumulo.core.util.LocalityGroupUtil;
 import org.apache.accumulo.core.util.LocalityGroupUtil.LocalityGroupConfigurationError;
 import org.apache.accumulo.core.util.Retry;
@@ -69,6 +66,8 @@
 import org.apache.thrift.transport.TTransport;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.net.HostAndPort;
+
 /**
  * Provides a class for administering the accumulo instance
  */
@@ -137,9 +136,9 @@
 
     var log = LoggerFactory.getLogger(InstanceOperationsImpl.class);
 
-    Retry retry =
-        Retry.builder().infiniteRetries().retryAfter(25, MILLISECONDS).incrementBy(25, MILLISECONDS)
-            .maxWait(30, SECONDS).backOffFactor(1.5).logInterval(3, MINUTES).createRetry();
+    Retry retry = Retry.builder().infiniteRetries().retryAfter(Duration.ofMillis(25))
+        .incrementBy(Duration.ofMillis(25)).maxWait(Duration.ofSeconds(30)).backOffFactor(1.5)
+        .logInterval(Duration.ofMinutes(3)).createRetry();
 
     while (true) {
       try {
@@ -206,6 +205,13 @@
   }
 
   @Override
+  public Map<String,String> getSystemProperties()
+      throws AccumuloException, AccumuloSecurityException {
+    return ThriftClientTypes.CLIENT.execute(context,
+        client -> client.getSystemProperties(TraceUtil.traceInfo(), context.rpcCreds()));
+  }
+
+  @Override
   public List<String> getManagerLocations() {
     return context.getManagerLocations();
   }
@@ -297,7 +303,7 @@
   public List<ActiveCompaction> getActiveCompactions()
       throws AccumuloException, AccumuloSecurityException {
 
-    Map<String,List<HostAndPort>> compactors = ExternalCompactionUtil.getCompactorAddrs(context);
+    Map<String,Set<HostAndPort>> compactors = ExternalCompactionUtil.getCompactorAddrs(context);
     List<String> tservers = getTabletServers();
 
     int numThreads = Math.max(4, Math.min((tservers.size() + compactors.size()) / 10, 256));
@@ -344,8 +350,7 @@
 
   @Override
   public void ping(String tserver) throws AccumuloException {
-    try (
-        TTransport transport = createTransport(AddressUtil.parseAddress(tserver, false), context)) {
+    try (TTransport transport = createTransport(AddressUtil.parseAddress(tserver), context)) {
       Client client = createClient(ThriftClientTypes.TABLET_SERVER, transport);
       client.getTabletServerStatus(TraceUtil.traceInfo(), context.rpcCreds());
     } catch (TException e) {
@@ -360,7 +365,7 @@
           client -> client.waitForBalance(TraceUtil.traceInfo()));
     } catch (AccumuloSecurityException ex) {
       // should never happen
-      throw new RuntimeException("Unexpected exception thrown", ex);
+      throw new IllegalStateException("Unexpected exception thrown", ex);
     }
 
   }
@@ -382,12 +387,6 @@
   }
 
   @Override
-  @Deprecated(since = "2.1.0")
-  public String getInstanceID() {
-    return getInstanceId().canonical();
-  }
-
-  @Override
   public InstanceId getInstanceId() {
     return context.getInstanceID();
   }
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 0b1d496..3c92fa5 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
@@ -20,13 +20,12 @@
 
 import static com.google.common.base.Preconditions.checkArgument;
 import static java.nio.charset.StandardCharsets.UTF_8;
-import static java.util.concurrent.TimeUnit.MILLISECONDS;
-import static java.util.concurrent.TimeUnit.MINUTES;
 import static java.util.concurrent.TimeUnit.SECONDS;
 import static org.apache.accumulo.core.util.Validators.EXISTING_NAMESPACE_NAME;
 import static org.apache.accumulo.core.util.Validators.NEW_NAMESPACE_NAME;
 
 import java.nio.ByteBuffer;
+import java.time.Duration;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.ConcurrentModificationException;
@@ -245,9 +244,9 @@
     EXISTING_NAMESPACE_NAME.validate(namespace);
     checkArgument(mapMutator != null, "mapMutator is null");
 
-    Retry retry =
-        Retry.builder().infiniteRetries().retryAfter(25, MILLISECONDS).incrementBy(25, MILLISECONDS)
-            .maxWait(30, SECONDS).backOffFactor(1.5).logInterval(3, MINUTES).createRetry();
+    Retry retry = Retry.builder().infiniteRetries().retryAfter(Duration.ofMillis(25))
+        .incrementBy(Duration.ofMillis(25)).maxWait(Duration.ofSeconds(30)).backOffFactor(1.5)
+        .logInterval(Duration.ofMinutes(3)).createRetry();
 
     while (true) {
       try {
@@ -338,7 +337,8 @@
   public Map<String,String> namespaceIdMap() {
     return Namespaces.getNameToIdMap(context).entrySet().stream()
         .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().canonical(), (v1, v2) -> {
-          throw new RuntimeException(String.format("Duplicate key for values %s and %s", v1, v2));
+          throw new IllegalStateException(
+              String.format("Duplicate key for values %s and %s", v1, v2));
         }, TreeMap::new));
   }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/OfflineIterator.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/OfflineIterator.java
index a03cc81..9200591 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/OfflineIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/OfflineIterator.java
@@ -18,13 +18,14 @@
  */
 package org.apache.accumulo.core.clientImpl;
 
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.FILES;
 import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.LOCATION;
 import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.PREV_ROW;
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
 
 import java.io.IOException;
+import java.io.UncheckedIOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -59,14 +60,12 @@
 import org.apache.accumulo.core.iteratorsImpl.system.SystemIteratorUtil;
 import org.apache.accumulo.core.manager.state.tables.TableState;
 import org.apache.accumulo.core.metadata.StoredTabletFile;
-import org.apache.accumulo.core.metadata.TabletFile;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata;
 import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
 import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.ColumnVisibility;
-import org.apache.accumulo.core.spi.common.ServiceEnvironment;
 import org.apache.accumulo.core.util.LocalityGroupUtil;
 import org.apache.accumulo.core.volume.VolumeConfiguration;
 import org.apache.hadoop.conf.Configuration;
@@ -94,12 +93,6 @@
       this.sampleConf = samplerConf;
     }
 
-    @Deprecated(since = "2.0.0")
-    @Override
-    public AccumuloConfiguration getConfig() {
-      return conf;
-    }
-
     @Override
     public IteratorScope getIteratorScope() {
       return IteratorScope.scan;
@@ -118,12 +111,6 @@
     private final ArrayList<SortedKeyValueIterator<Key,Value>> topLevelIterators =
         new ArrayList<>();
 
-    @Deprecated(since = "2.0.0")
-    @Override
-    public void registerSideChannel(SortedKeyValueIterator<Key,Value> iter) {
-      topLevelIterators.add(iter);
-    }
-
     @Override
     public Authorizations getAuthorizations() {
       return authorizations;
@@ -157,12 +144,6 @@
           sampleConf);
     }
 
-    @Deprecated(since = "2.1.0")
-    @Override
-    public ServiceEnvironment getServiceEnv() {
-      return new ClientServiceEnvironmentImpl(context);
-    }
-
     @Override
     public PluginEnvironment getPluginEnv() {
       return new ClientServiceEnvironmentImpl(context);
@@ -205,11 +186,10 @@
         nextTablet();
       }
 
-    } catch (Exception e) {
-      if (e instanceof RuntimeException) {
-        throw (RuntimeException) e;
-      }
-      throw new RuntimeException(e);
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    } catch (AccumuloException | AccumuloSecurityException | TableNotFoundException e) {
+      throw new IllegalStateException(e);
     }
   }
 
@@ -232,8 +212,10 @@
       }
 
       return ret;
-    } catch (Exception e) {
-      throw new RuntimeException(e);
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    } catch (AccumuloException | AccumuloSecurityException | TableNotFoundException e) {
+      throw new IllegalStateException(e);
     }
   }
 
@@ -329,11 +311,11 @@
     if (scannerSamplerConfigImpl != null && !scannerSamplerConfigImpl.equals(samplerConfImpl)) {
       throw new SampleNotPresentException();
     }
-    for (TabletFile file : absFiles) {
+    for (StoredTabletFile file : absFiles) {
       var cs = CryptoFactoryLoader.getServiceForClientWithTable(systemConf, tableConf, tableId);
-      FileSystem fs = VolumeConfiguration.fileSystemForPath(file.getPathStr(), conf);
+      FileSystem fs = VolumeConfiguration.fileSystemForPath(file.getNormalizedPathStr(), conf);
       FileSKVIterator reader = FileOperations.getInstance().newReaderBuilder()
-          .forFile(file.getPathStr(), fs, conf, cs).withTableConfiguration(tableCC).build();
+          .forFile(file, fs, conf, cs).withTableConfiguration(tableCC).build();
       if (scannerSamplerConfigImpl != null) {
         reader = reader.getSample(scannerSamplerConfigImpl);
         if (reader == null) {
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ReplicationOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ReplicationOperationsImpl.java
deleted file mode 100644
index 49aa1f5..0000000
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ReplicationOperationsImpl.java
+++ /dev/null
@@ -1,150 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.clientImpl;
-
-import static java.util.Objects.requireNonNull;
-
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Map.Entry;
-import java.util.Set;
-
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.BatchScanner;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.admin.ReplicationOperations;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.ReplicationSection;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LogColumnFamily;
-import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
-import org.apache.accumulo.core.tabletserver.log.LogEntry;
-import org.apache.accumulo.core.trace.TraceUtil;
-import org.apache.accumulo.core.trace.thrift.TInfo;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-@Deprecated
-public class ReplicationOperationsImpl implements ReplicationOperations {
-  private static final Logger log = LoggerFactory.getLogger(ReplicationOperationsImpl.class);
-
-  private final ClientContext context;
-
-  public ReplicationOperationsImpl(ClientContext context) {
-    this.context = requireNonNull(context);
-  }
-
-  @Override
-  public void addPeer(final String name, final String replicaType)
-      throws AccumuloException, AccumuloSecurityException {
-    context.instanceOperations().setProperty(
-        Property.REPLICATION_PEERS.getKey() + requireNonNull(name), requireNonNull(replicaType));
-  }
-
-  @Override
-  public void removePeer(final String name) throws AccumuloException, AccumuloSecurityException {
-    context.instanceOperations()
-        .removeProperty(Property.REPLICATION_PEERS.getKey() + requireNonNull(name));
-  }
-
-  @Override
-  public void drain(String tableName)
-      throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    drain(tableName, referencedFiles(requireNonNull(tableName)));
-  }
-
-  @Override
-  public void drain(final String tableName, final Set<String> wals)
-      throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    requireNonNull(tableName);
-
-    final TInfo tinfo = TraceUtil.traceInfo();
-    final TCredentials rpcCreds = context.rpcCreds();
-
-    // Ask the manager if the table is fully replicated given these WALs, but don't poll inside the
-    // manager
-    boolean drained = false;
-    while (!drained) {
-      drained = getManagerDrain(tinfo, rpcCreds, tableName, wals);
-
-      if (!drained) {
-        try {
-          Thread.sleep(1000);
-        } catch (InterruptedException e) {
-          Thread.currentThread().interrupt();
-          throw new RuntimeException("Thread interrupted", e);
-        }
-      }
-    }
-  }
-
-  protected boolean getManagerDrain(final TInfo tinfo, final TCredentials rpcCreds,
-      final String tableName, final Set<String> wals)
-      throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    return ThriftClientTypes.MANAGER.execute(context,
-        client -> client.drainReplicationTable(tinfo, rpcCreds, tableName, wals));
-  }
-
-  @Override
-  public Set<String> referencedFiles(String tableName) throws TableNotFoundException {
-    log.debug("Collecting referenced files for replication of table {}", tableName);
-    TableId tableId = context.getTableId(tableName);
-    log.debug("Found id of {} for name {}", tableId, tableName);
-
-    // Get the WALs currently referenced by the table
-    BatchScanner metaBs = context.createBatchScanner(MetadataTable.NAME, Authorizations.EMPTY, 4);
-    metaBs.setRanges(Collections.singleton(TabletsSection.getRange(tableId)));
-    metaBs.fetchColumnFamily(LogColumnFamily.NAME);
-    Set<String> wals = new HashSet<>();
-    try {
-      for (Entry<Key,Value> entry : metaBs) {
-        LogEntry logEntry = LogEntry.fromMetaWalEntry(entry);
-        wals.add(new Path(logEntry.filename).toString());
-      }
-    } finally {
-      metaBs.close();
-    }
-
-    // And the WALs that need to be replicated for this table
-    metaBs = context.createBatchScanner(MetadataTable.NAME, Authorizations.EMPTY, 4);
-    metaBs.setRanges(Collections.singleton(ReplicationSection.getRange()));
-    metaBs.fetchColumnFamily(ReplicationSection.COLF);
-    try {
-      Text buffer = new Text();
-      for (Entry<Key,Value> entry : metaBs) {
-        if (tableId.equals(ReplicationSection.getTableId(entry.getKey()))) {
-          ReplicationSection.getFile(entry.getKey(), buffer);
-          wals.add(buffer.toString());
-        }
-      }
-    } finally {
-      metaBs.close();
-    }
-    return wals;
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/RootTabletLocator.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/RootTabletLocator.java
index 28aa5b9..a24a236 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/RootTabletLocator.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/RootTabletLocator.java
@@ -18,10 +18,10 @@
  */
 package org.apache.accumulo.core.clientImpl;
 
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static java.util.concurrent.TimeUnit.SECONDS;
 import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.LOCATION;
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
 
 import java.util.Collection;
 import java.util.Collections;
@@ -56,11 +56,11 @@
       Map<String,TabletServerMutations<T>> binnedMutations, List<T> failures) {
     TabletLocation rootTabletLocation = getRootTabletLocation(context);
     if (rootTabletLocation != null) {
-      TabletServerMutations<T> tsm = new TabletServerMutations<>(rootTabletLocation.tablet_session);
+      var tsm = new TabletServerMutations<T>(rootTabletLocation.getTserverSession());
       for (T mutation : mutations) {
         tsm.addMutation(RootTable.EXTENT, mutation);
       }
-      binnedMutations.put(rootTabletLocation.tablet_location, tsm);
+      binnedMutations.put(rootTabletLocation.getTserverLocation(), tsm);
     } else {
       failures.addAll(mutations);
     }
@@ -73,7 +73,7 @@
     TabletLocation rootTabletLocation = getRootTabletLocation(context);
     if (rootTabletLocation != null) {
       for (Range range : ranges) {
-        TabletLocatorImpl.addRange(binnedRanges, rootTabletLocation.tablet_location,
+        TabletLocatorImpl.addRange(binnedRanges, rootTabletLocation.getTserverLocation(),
             RootTable.EXTENT, range);
       }
       return Collections.emptyList();
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerIterator.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerIterator.java
index 182b912..1f065bb 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerIterator.java
@@ -27,11 +27,15 @@
 import java.util.concurrent.Future;
 import java.util.concurrent.atomic.AtomicBoolean;
 
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.SampleNotPresentException;
 import org.apache.accumulo.core.client.ScannerBase.ConsistencyLevel;
 import org.apache.accumulo.core.client.TableDeletedException;
+import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.TableOfflineException;
 import org.apache.accumulo.core.clientImpl.ThriftScanner.ScanState;
+import org.apache.accumulo.core.clientImpl.ThriftScanner.ScanTimedOutException;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.KeyValue;
 import org.apache.accumulo.core.data.Range;
@@ -143,7 +147,8 @@
     readAheadOperation = context.submitScannerReadAheadTask(this::readBatch);
   }
 
-  private List<KeyValue> readBatch() throws Exception {
+  private List<KeyValue> readBatch() throws ScanTimedOutException, AccumuloException,
+      AccumuloSecurityException, TableNotFoundException {
 
     List<KeyValue> batch;
 
@@ -176,11 +181,10 @@
       }
     } catch (ExecutionException ee) {
       wrapExecutionException(ee);
-      throw new RuntimeException(ee);
-    } catch (RuntimeException e) {
-      throw e;
-    } catch (Exception e) {
-      throw new RuntimeException(e);
+      throw new IllegalStateException(ee);
+    } catch (AccumuloException | AccumuloSecurityException | TableNotFoundException
+        | ScanTimedOutException | InterruptedException e) {
+      throw new IllegalStateException(e);
     }
 
     if (!nextBatch.isEmpty()) {
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/SyncingTabletLocator.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/SyncingTabletLocator.java
index 8f6daed..df40a21 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/SyncingTabletLocator.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/SyncingTabletLocator.java
@@ -21,7 +21,7 @@
 import java.util.Collection;
 import java.util.List;
 import java.util.Map;
-import java.util.concurrent.Callable;
+import java.util.function.Supplier;
 
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -31,8 +31,6 @@
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.hadoop.io.Text;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /**
  * Syncs itself with the static collection of TabletLocators, so that when the server clears it, it
@@ -40,19 +38,13 @@
  * using SyncingTabletLocator.
  */
 public class SyncingTabletLocator extends TabletLocator {
-  private static final Logger log = LoggerFactory.getLogger(SyncingTabletLocator.class);
 
   private volatile TabletLocator locator;
-  private final Callable<TabletLocator> getLocatorFunction;
+  private final Supplier<TabletLocator> getLocatorFunction;
 
-  public SyncingTabletLocator(Callable<TabletLocator> getLocatorFunction) {
+  public SyncingTabletLocator(Supplier<TabletLocator> getLocatorFunction) {
     this.getLocatorFunction = getLocatorFunction;
-    try {
-      this.locator = getLocatorFunction.call();
-    } catch (Exception e) {
-      log.error("Problem obtaining TabletLocator", e);
-      throw new RuntimeException(e);
-    }
+    this.locator = getLocatorFunction.get();
   }
 
   public SyncingTabletLocator(final ClientContext context, final TableId tableId) {
@@ -64,12 +56,7 @@
     if (!loc.isValid()) {
       synchronized (this) {
         if (locator == loc) {
-          try {
-            loc = locator = getLocatorFunction.call();
-          } catch (Exception e) {
-            log.error("Problem obtaining TabletLocator", e);
-            throw new RuntimeException(e);
-          }
+          loc = locator = getLocatorFunction.get();
         }
       }
     }
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 129cb6a..6b94fd8 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
@@ -19,15 +19,15 @@
 package org.apache.accumulo.core.clientImpl;
 
 import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 import static java.nio.charset.StandardCharsets.UTF_8;
 import static java.util.Objects.requireNonNull;
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
-import static java.util.concurrent.TimeUnit.MINUTES;
 import static java.util.concurrent.TimeUnit.SECONDS;
 import static java.util.stream.Collectors.toSet;
 import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.LOCATION;
 import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.PREV_ROW;
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 import static org.apache.accumulo.core.util.Validators.EXISTING_TABLE_NAME;
 import static org.apache.accumulo.core.util.Validators.NEW_TABLE_NAME;
 
@@ -36,7 +36,7 @@
 import java.io.IOException;
 import java.io.InputStreamReader;
 import java.nio.ByteBuffer;
-import java.security.SecureRandom;
+import java.time.Duration;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -119,9 +119,8 @@
 import org.apache.accumulo.core.manager.thrift.FateOperation;
 import org.apache.accumulo.core.manager.thrift.FateService;
 import org.apache.accumulo.core.manager.thrift.ManagerClientService;
-import org.apache.accumulo.core.metadata.MetadataServicer;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
+import org.apache.accumulo.core.metadata.schema.TabletDeletedException;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata.LocationType;
@@ -132,10 +131,9 @@
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.summary.SummarizerConfigurationUtil;
 import org.apache.accumulo.core.summary.SummaryCollection;
+import org.apache.accumulo.core.tablet.thrift.TabletManagementClientService;
 import org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException;
-import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
 import org.apache.accumulo.core.trace.TraceUtil;
-import org.apache.accumulo.core.util.HostAndPort;
 import org.apache.accumulo.core.util.LocalityGroupUtil;
 import org.apache.accumulo.core.util.LocalityGroupUtil.LocalityGroupConfigurationError;
 import org.apache.accumulo.core.util.MapCounter;
@@ -156,11 +154,10 @@
 
 import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
+import com.google.common.net.HostAndPort;
 
 public class TableOperationsImpl extends TableOperationsHelper {
 
-  private static final SecureRandom random = new SecureRandom();
-
   public static final String PROPERTY_EXCLUDE_PREFIX = "!";
   public static final String COMPACTION_CANCELED_MSG = "Compaction canceled";
   public static final String TABLE_DELETED_MSG = "Table is being deleted";
@@ -198,7 +195,8 @@
   public boolean exists(String tableName) {
     EXISTING_TABLE_NAME.validate(tableName);
 
-    if (tableName.equals(MetadataTable.NAME) || tableName.equals(RootTable.NAME)) {
+    if (tableName.equals(AccumuloTable.METADATA.tableName())
+        || tableName.equals(AccumuloTable.ROOT.tableName())) {
       return true;
     }
 
@@ -531,7 +529,7 @@
         }
       }
     } catch (InterruptedException e) {
-      throw new RuntimeException(e);
+      throw new IllegalStateException(e);
     } finally {
       executor.shutdown();
     }
@@ -562,26 +560,26 @@
           continue;
         }
 
-        HostAndPort address = HostAndPort.fromString(tl.tablet_location);
+        HostAndPort address = HostAndPort.fromString(tl.getTserverLocation());
 
         try {
-          TabletClientService.Client client =
-              ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, address, context);
+          TabletManagementClientService.Client client =
+              ThriftUtil.getClient(ThriftClientTypes.TABLET_MGMT, address, context);
           try {
 
             OpTimer timer = null;
 
             if (log.isTraceEnabled()) {
               log.trace("tid={} Splitting tablet {} on {} at {}", Thread.currentThread().getId(),
-                  tl.tablet_extent, address, split);
+                  tl.getExtent(), address, split);
               timer = new OpTimer().start();
             }
 
-            client.splitTablet(TraceUtil.traceInfo(), context.rpcCreds(),
-                tl.tablet_extent.toThrift(), TextUtil.getByteBuffer(split));
+            client.splitTablet(TraceUtil.traceInfo(), context.rpcCreds(), tl.getExtent().toThrift(),
+                TextUtil.getByteBuffer(split));
 
             // just split it, might as well invalidate it in the cache
-            tabLocator.invalidateCache(tl.tablet_extent);
+            tabLocator.invalidateCache(tl.getExtent());
 
             if (timer != null) {
               timer.stop();
@@ -606,10 +604,10 @@
                 + " Seen {} failures.", split, env.tableName, locationFailures);
           }
 
-          tabLocator.invalidateCache(tl.tablet_extent);
+          tabLocator.invalidateCache(tl.getExtent());
           continue;
         } catch (TException e) {
-          tabLocator.invalidateCache(context, tl.tablet_location);
+          tabLocator.invalidateCache(context, tl.getTserverLocation());
           continue;
         }
 
@@ -665,37 +663,22 @@
 
   private List<Text> _listSplits(String tableName)
       throws TableNotFoundException, AccumuloSecurityException {
+
     TableId tableId = context.getTableId(tableName);
-    TreeMap<KeyExtent,String> tabletLocations = new TreeMap<>();
+
     while (true) {
-      try {
-        tabletLocations.clear();
-        // the following method throws AccumuloException for some conditions that should be retried
-        MetadataServicer.forTableId(context, tableId).getTabletLocations(tabletLocations);
-        break;
-      } catch (AccumuloSecurityException ase) {
-        throw ase;
-      } catch (Exception e) {
+      try (TabletsMetadata tabletsMetadata = context.getAmple().readTablets().forTable(tableId)
+          .fetch(PREV_ROW).checkConsistency().build()) {
+        return tabletsMetadata.stream().map(tm -> tm.getExtent().endRow()).filter(Objects::nonNull)
+            .collect(Collectors.toList());
+      } catch (TabletDeletedException tde) {
+        // see if the table was deleted
         context.requireTableExists(tableId, tableName);
-
-        if (e instanceof RuntimeException && e.getCause() instanceof AccumuloSecurityException) {
-          throw (AccumuloSecurityException) e.getCause();
-        }
-
-        log.info("{} ... retrying ...", e, e);
+        log.debug("A merge happened while trying to list splits for {} {}, retrying ", tableName,
+            tableId, tde);
         sleepUninterruptibly(3, SECONDS);
       }
     }
-
-    ArrayList<Text> endRows = new ArrayList<>(tabletLocations.size());
-    for (KeyExtent ke : tabletLocations.keySet()) {
-      if (ke.endRow() != null) {
-        endRows.add(ke.endRow());
-      }
-    }
-
-    return endRows;
-
   }
 
   /**
@@ -860,8 +843,6 @@
       }
     }
 
-    ensureStrategyCanLoad(tableName, config);
-
     if (!UserCompactionUtils.isDefault(config.getConfigurer())) {
       if (!testClassLoad(tableName, config.getConfigurer().getClassName(),
           CompactionConfigurer.class.getName())) {
@@ -902,19 +883,6 @@
     }
   }
 
-  @SuppressWarnings("removal")
-  private void ensureStrategyCanLoad(String tableName, CompactionConfig config)
-      throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
-    // Make sure the specified compaction strategy exists on a tabletserver
-    if (!CompactionStrategyConfigUtil.isDefault(config.getCompactionStrategy())) {
-      if (!testClassLoad(tableName, config.getCompactionStrategy().getClassName(),
-          "org.apache.accumulo.tserver.compaction.CompactionStrategy")) {
-        throw new AccumuloException("TabletServer could not load CompactionStrategy class "
-            + config.getCompactionStrategy().getClassName());
-      }
-    }
-  }
-
   @Override
   public void cancelCompaction(String tableName)
       throws AccumuloSecurityException, TableNotFoundException, AccumuloException {
@@ -1053,9 +1021,9 @@
     EXISTING_TABLE_NAME.validate(tableName);
     checkArgument(mapMutator != null, "mapMutator is null");
 
-    Retry retry =
-        Retry.builder().infiniteRetries().retryAfter(25, MILLISECONDS).incrementBy(25, MILLISECONDS)
-            .maxWait(30, SECONDS).backOffFactor(1.5).logInterval(3, MINUTES).createRetry();
+    Retry retry = Retry.builder().infiniteRetries().retryAfter(Duration.ofMillis(25))
+        .incrementBy(Duration.ofMillis(25)).maxWait(Duration.ofSeconds(30)).backOffFactor(1.5)
+        .logInterval(Duration.ofMinutes(3)).createRetry();
 
     while (true) {
       try {
@@ -1261,7 +1229,7 @@
 
       log.warn("Unable to locate bins for specified range. Retrying.");
       // sleep randomly between 100 and 200ms
-      sleepUninterruptibly(100 + random.nextInt(100), MILLISECONDS);
+      sleepUninterruptibly(100 + RANDOM.get().nextInt(100), MILLISECONDS);
       binnedRanges.clear();
       tl.invalidateCache();
     }
@@ -1327,34 +1295,6 @@
     return ret;
   }
 
-  @Override
-  @Deprecated(since = "2.0.0")
-  public void importDirectory(String tableName, String dir, String failureDir, boolean setTime)
-      throws IOException, AccumuloSecurityException, TableNotFoundException, AccumuloException {
-    EXISTING_TABLE_NAME.validate(tableName);
-    checkArgument(dir != null, "dir is null");
-    checkArgument(failureDir != null, "failureDir is null");
-
-    // check for table existence
-    context.getTableId(tableName);
-    Path dirPath = checkPath(dir, "Bulk", "");
-    Path failPath = checkPath(failureDir, "Bulk", "failure");
-
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(UTF_8)),
-        ByteBuffer.wrap(dirPath.toString().getBytes(UTF_8)),
-        ByteBuffer.wrap(failPath.toString().getBytes(UTF_8)),
-        ByteBuffer.wrap((setTime + "").getBytes(UTF_8)));
-    Map<String,String> opts = new HashMap<>();
-
-    try {
-      doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_BULK_IMPORT,
-          args, opts);
-    } catch (TableExistsException e) {
-      // should not happen
-      throw new AssertionError(e);
-    }
-  }
-
   private void waitForTableStateTransition(TableId tableId, TableState expectedState)
       throws AccumuloException, TableNotFoundException {
     Text startRow = null;
@@ -1381,9 +1321,6 @@
         range = new Range(startRow, lastRow);
       }
 
-      TabletsMetadata tablets = TabletsMetadata.builder(context).scanMetadataTable()
-          .overRange(range).fetch(LOCATION, PREV_ROW).build();
-
       KeyExtent lastExtent = null;
 
       int total = 0;
@@ -1392,34 +1329,38 @@
       Text continueRow = null;
       MapCounter<String> serverCounts = new MapCounter<>();
 
-      for (TabletMetadata tablet : tablets) {
-        total++;
-        Location loc = tablet.getLocation();
+      try (TabletsMetadata tablets = TabletsMetadata.builder(context).scanMetadataTable()
+          .overRange(range).fetch(LOCATION, PREV_ROW).build()) {
 
-        if ((expectedState == TableState.ONLINE
-            && (loc == null || loc.getType() == LocationType.FUTURE))
-            || (expectedState == TableState.OFFLINE && loc != null)) {
-          if (continueRow == null) {
-            continueRow = tablet.getExtent().toMetaRow();
+        for (TabletMetadata tablet : tablets) {
+          total++;
+          Location loc = tablet.getLocation();
+
+          if ((expectedState == TableState.ONLINE
+              && (loc == null || loc.getType() == LocationType.FUTURE))
+              || (expectedState == TableState.OFFLINE && loc != null)) {
+            if (continueRow == null) {
+              continueRow = tablet.getExtent().toMetaRow();
+            }
+            waitFor++;
+            lastRow = tablet.getExtent().toMetaRow();
+
+            if (loc != null) {
+              serverCounts.increment(loc.getHostPortSession(), 1);
+            }
           }
-          waitFor++;
-          lastRow = tablet.getExtent().toMetaRow();
 
-          if (loc != null) {
-            serverCounts.increment(loc.getHostPortSession(), 1);
+          if (!tablet.getExtent().tableId().equals(tableId)) {
+            throw new AccumuloException(
+                "Saw unexpected table Id " + tableId + " " + tablet.getExtent());
           }
-        }
 
-        if (!tablet.getExtent().tableId().equals(tableId)) {
-          throw new AccumuloException(
-              "Saw unexpected table Id " + tableId + " " + tablet.getExtent());
-        }
+          if (lastExtent != null && !tablet.getExtent().isPreviousExtent(lastExtent)) {
+            holes++;
+          }
 
-        if (lastExtent != null && !tablet.getExtent().isPreviousExtent(lastExtent)) {
-          holes++;
+          lastExtent = tablet.getExtent();
         }
-
-        lastExtent = tablet.getExtent();
       }
 
       if (continueRow != null) {
@@ -1539,7 +1480,8 @@
   public Map<String,String> tableIdMap() {
     return context.getTableNameToIdMap().entrySet().stream()
         .collect(Collectors.toMap(Entry::getKey, e -> e.getValue().canonical(), (v1, v2) -> {
-          throw new RuntimeException(String.format("Duplicate key for values %s and %s", v1, v2));
+          throw new IllegalStateException(
+              String.format("Duplicate key for values %s and %s", v1, v2));
         }, TreeMap::new));
   }
 
@@ -1563,7 +1505,7 @@
         // this operation may us a lot of memory... its likely that connections to tabletservers
         // hosting metadata tablets will be cached, so do not use cached
         // connections
-        pair = ThriftClientTypes.CLIENT.getTabletServerConnection(context, false);
+        pair = ThriftClientTypes.CLIENT.getThriftServerConnection(context, false);
         diskUsages = pair.getSecond().getDiskUsage(tableNames, context.rpcCreds());
       } catch (ThriftTableOperationException e) {
         switch (e.getType()) {
@@ -1886,7 +1828,7 @@
           List<Range> prev =
               groupedByTablets.put(tabletId, Collections.unmodifiableList(entry2.getValue()));
           if (prev != null) {
-            throw new RuntimeException(
+            throw new IllegalStateException(
                 "Unexpected : tablet at multiple locations : " + location + " " + tabletId);
           }
         }
@@ -1945,9 +1887,9 @@
 
     locator.invalidateCache();
 
-    Retry retry = Retry.builder().infiniteRetries().retryAfter(100, MILLISECONDS)
-        .incrementBy(100, MILLISECONDS).maxWait(2, SECONDS).backOffFactor(1.5)
-        .logInterval(3, MINUTES).createRetry();
+    Retry retry = Retry.builder().infiniteRetries().retryAfter(Duration.ofMillis(100))
+        .incrementBy(Duration.ofMillis(100)).maxWait(Duration.ofSeconds(2)).backOffFactor(1.5)
+        .logInterval(Duration.ofMinutes(3)).createRetry();
 
     while (!locator.binRanges(context, rangeList, binnedRanges).isEmpty()) {
       context.requireTableExists(tableId, tableName);
@@ -1958,7 +1900,7 @@
             String.format("locating tablets in table %s(%s) for %d ranges", tableName, tableId,
                 rangeList.size()));
       } catch (InterruptedException e) {
-        throw new RuntimeException(e);
+        throw new IllegalStateException(e);
       }
       locator.invalidateCache();
     }
@@ -2119,8 +2061,11 @@
   @Override
   public TimeType getTimeType(final String tableName) throws TableNotFoundException {
     TableId tableId = context.getTableId(tableName);
-    Optional<TabletMetadata> tabletMetadata = context.getAmple().readTablets().forTable(tableId)
-        .fetch(TabletMetadata.ColumnType.TIME).checkConsistency().build().stream().findFirst();
+    Optional<TabletMetadata> tabletMetadata;
+    try (TabletsMetadata tabletsMetadata = context.getAmple().readTablets().forTable(tableId)
+        .fetch(TabletMetadata.ColumnType.TIME).checkConsistency().build()) {
+      tabletMetadata = tabletsMetadata.stream().findFirst();
+    }
     TabletMetadata timeData =
         tabletMetadata.orElseThrow(() -> new IllegalStateException("Failed to retrieve TimeType"));
     return timeData.getTime().getType();
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletLocator.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletLocator.java
index b8e4c53..9c528c3 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletLocator.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletLocator.java
@@ -25,6 +25,7 @@
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
 
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -34,9 +35,8 @@
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.metadata.MetadataLocationObtainer;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.RootTable;
 import org.apache.accumulo.core.singletons.SingletonManager;
 import org.apache.accumulo.core.singletons.SingletonService;
 import org.apache.accumulo.core.util.Interner;
@@ -140,14 +140,15 @@
     if (tl == null) {
       MetadataLocationObtainer mlo = new MetadataLocationObtainer();
 
-      if (RootTable.ID.equals(tableId)) {
+      if (AccumuloTable.ROOT.tableId().equals(tableId)) {
         tl = new RootTabletLocator(new ZookeeperLockChecker(context));
-      } else if (MetadataTable.ID.equals(tableId)) {
-        tl = new TabletLocatorImpl(MetadataTable.ID, getLocator(context, RootTable.ID), mlo,
+      } else if (AccumuloTable.METADATA.tableId().equals(tableId)) {
+        tl = new TabletLocatorImpl(AccumuloTable.METADATA.tableId(),
+            getLocator(context, AccumuloTable.ROOT.tableId()), mlo,
             new ZookeeperLockChecker(context));
       } else {
-        tl = new TabletLocatorImpl(tableId, getLocator(context, MetadataTable.ID), mlo,
-            new ZookeeperLockChecker(context));
+        tl = new TabletLocatorImpl(tableId, getLocator(context, AccumuloTable.METADATA.tableId()),
+            mlo, new ZookeeperLockChecker(context));
       }
       locators.put(key, tl);
     }
@@ -194,54 +195,53 @@
     }
   }
 
-  public static class TabletLocation implements Comparable<TabletLocation> {
+  public static class TabletLocation {
     private static final Interner<String> interner = new Interner<>();
 
-    public final KeyExtent tablet_extent;
-    public final String tablet_location;
-    public final String tablet_session;
+    private final KeyExtent tablet_extent;
+    private final String tserverLocation;
+    private final String tserverSession;
 
     public TabletLocation(KeyExtent tablet_extent, String tablet_location, String session) {
       checkArgument(tablet_extent != null, "tablet_extent is null");
       checkArgument(tablet_location != null, "tablet_location is null");
       checkArgument(session != null, "session is null");
       this.tablet_extent = tablet_extent;
-      this.tablet_location = interner.intern(tablet_location);
-      this.tablet_session = interner.intern(session);
+      this.tserverLocation = interner.intern(tablet_location);
+      this.tserverSession = interner.intern(session);
     }
 
     @Override
     public boolean equals(Object o) {
       if (o instanceof TabletLocation) {
         TabletLocation otl = (TabletLocation) o;
-        return tablet_extent.equals(otl.tablet_extent)
-            && tablet_location.equals(otl.tablet_location)
-            && tablet_session.equals(otl.tablet_session);
+        return getExtent().equals(otl.getExtent())
+            && getTserverLocation().equals(otl.getTserverLocation())
+            && getTserverSession().equals(otl.getTserverSession());
       }
       return false;
     }
 
     @Override
     public int hashCode() {
-      throw new UnsupportedOperationException(
-          "hashcode is not implemented for class " + this.getClass());
+      return Objects.hash(getExtent(), tserverLocation, tserverSession);
     }
 
     @Override
     public String toString() {
-      return "(" + tablet_extent + "," + tablet_location + "," + tablet_session + ")";
+      return "(" + getExtent() + "," + getTserverLocation() + "," + getTserverSession() + ")";
     }
 
-    @Override
-    public int compareTo(TabletLocation o) {
-      int result = tablet_extent.compareTo(o.tablet_extent);
-      if (result == 0) {
-        result = tablet_location.compareTo(o.tablet_location);
-        if (result == 0) {
-          result = tablet_session.compareTo(o.tablet_session);
-        }
-      }
-      return result;
+    public KeyExtent getExtent() {
+      return tablet_extent;
+    }
+
+    public String getTserverLocation() {
+      return tserverLocation;
+    }
+
+    public String getTserverSession() {
+      return tserverSession;
     }
   }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletLocatorImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletLocatorImpl.java
index 02844db..dafada9 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletLocatorImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletLocatorImpl.java
@@ -18,9 +18,9 @@
  */
 package org.apache.accumulo.core.clientImpl;
 
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static java.util.concurrent.TimeUnit.SECONDS;
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
 
 import java.util.ArrayList;
 import java.util.Collection;
@@ -125,7 +125,7 @@
         return null;
       }
 
-      Pair<String,String> lock = new Pair<>(tl.tablet_location, tl.tablet_session);
+      Pair<String,String> lock = new Pair<>(tl.getTserverLocation(), tl.getTserverSession());
 
       if (okLocks.contains(lock)) {
         return tl;
@@ -135,14 +135,14 @@
         return null;
       }
 
-      if (lockChecker.isLockHeld(tl.tablet_location, tl.tablet_session)) {
+      if (lockChecker.isLockHeld(tl.getTserverLocation(), tl.getTserverSession())) {
         okLocks.add(lock);
         return tl;
       }
 
       if (log.isTraceEnabled()) {
-        log.trace("Tablet server {} {} no longer holds its lock", tl.tablet_location,
-            tl.tablet_session);
+        log.trace("Tablet server {} {} no longer holds its lock", tl.getTserverLocation(),
+            tl.getTserverSession());
       }
 
       invalidLocks.add(lock);
@@ -244,22 +244,22 @@
   private <T extends Mutation> boolean addMutation(
       Map<String,TabletServerMutations<T>> binnedMutations, T mutation, TabletLocation tl,
       LockCheckerSession lcSession) {
-    TabletServerMutations<T> tsm = binnedMutations.get(tl.tablet_location);
+    TabletServerMutations<T> tsm = binnedMutations.get(tl.getTserverLocation());
 
     if (tsm == null) {
       // do lock check once per tserver here to make binning faster
       boolean lockHeld = lcSession.checkLock(tl) != null;
       if (lockHeld) {
-        tsm = new TabletServerMutations<>(tl.tablet_session);
-        binnedMutations.put(tl.tablet_location, tsm);
+        tsm = new TabletServerMutations<>(tl.getTserverSession());
+        binnedMutations.put(tl.getTserverLocation(), tsm);
       } else {
         return false;
       }
     }
 
     // its possible the same tserver could be listed with different sessions
-    if (tsm.getSession().equals(tl.tablet_session)) {
-      tsm.addMutation(tl.tablet_extent, mutation);
+    if (tsm.getSession().equals(tl.getTserverSession())) {
+      tsm.addMutation(tl.getExtent(), mutation);
       return true;
     }
 
@@ -269,10 +269,10 @@
   static boolean isContiguous(List<TabletLocation> tabletLocations) {
 
     Iterator<TabletLocation> iter = tabletLocations.iterator();
-    KeyExtent prevExtent = iter.next().tablet_extent;
+    KeyExtent prevExtent = iter.next().getExtent();
 
     while (iter.hasNext()) {
-      KeyExtent currExtent = iter.next().tablet_extent;
+      KeyExtent currExtent = iter.next().getExtent();
 
       if (!currExtent.isPreviousExtent(prevExtent)) {
         return false;
@@ -323,14 +323,14 @@
 
       tabletLocations.add(tl);
 
-      while (tl.tablet_extent.endRow() != null
-          && !range.afterEndKey(new Key(tl.tablet_extent.endRow()).followingKey(PartialKey.ROW))) {
+      while (tl.getExtent().endRow() != null
+          && !range.afterEndKey(new Key(tl.getExtent().endRow()).followingKey(PartialKey.ROW))) {
         if (useCache) {
-          Text row = new Text(tl.tablet_extent.endRow());
+          Text row = new Text(tl.getExtent().endRow());
           row.append(new byte[] {0}, 0, 1);
           tl = lcSession.checkLock(locateTabletInCache(row));
         } else {
-          tl = _locateTablet(context, tl.tablet_extent.endRow(), true, false, false, lcSession);
+          tl = _locateTablet(context, tl.getExtent().endRow(), true, false, false, lcSession);
         }
 
         if (tl == null) {
@@ -349,7 +349,8 @@
       // then after that merges and splits happen.
       if (isContiguous(tabletLocations)) {
         for (TabletLocation tl2 : tabletLocations) {
-          TabletLocatorImpl.addRange(binnedRanges, tl2.tablet_location, tl2.tablet_extent, range);
+          TabletLocatorImpl.addRange(binnedRanges, tl2.getTserverLocation(), tl2.getExtent(),
+              range);
         }
       } else {
         failures.add(range);
@@ -454,8 +455,8 @@
     wLock.lock();
     try {
       for (TabletLocation cacheEntry : metaCache.values()) {
-        if (cacheEntry.tablet_location.equals(server)) {
-          badExtents.add(cacheEntry.tablet_extent);
+        if (cacheEntry.getTserverLocation().equals(server)) {
+          badExtents.add(cacheEntry.getExtent());
           invalidatedCount++;
         }
       }
@@ -516,7 +517,7 @@
       if (timer != null) {
         timer.stop();
         log.trace("tid={} Located tablet {} at {} in {}", Thread.currentThread().getId(),
-            (tl == null ? "null" : tl.tablet_extent), (tl == null ? "null" : tl.tablet_location),
+            (tl == null ? "null" : tl.getExtent()), (tl == null ? "null" : tl.getTserverLocation()),
             String.format("%.3f secs", timer.scale(SECONDS)));
       }
 
@@ -538,7 +539,7 @@
       while (locations != null && locations.getLocations().isEmpty()
           && locations.getLocationless().isEmpty()) {
         // try the next tablet, the current tablet does not have any tablets that overlap the row
-        Text er = ptl.tablet_extent.endRow();
+        Text er = ptl.getExtent().endRow();
         if (er != null && er.compareTo(lastTabletRow) < 0) {
           // System.out.println("er "+er+" ltr "+lastTabletRow);
           ptl = parent.locateTablet(context, er, true, retry);
@@ -563,20 +564,20 @@
       Text lastEndRow = null;
       for (TabletLocation tabletLocation : locations.getLocations()) {
 
-        KeyExtent ke = tabletLocation.tablet_extent;
+        KeyExtent ke = tabletLocation.getExtent();
         TabletLocation locToCache;
 
         // create new location if current prevEndRow == endRow
         if ((lastEndRow != null) && (ke.prevEndRow() != null)
             && ke.prevEndRow().equals(lastEndRow)) {
           locToCache = new TabletLocation(new KeyExtent(ke.tableId(), ke.endRow(), lastEndRow),
-              tabletLocation.tablet_location, tabletLocation.tablet_session);
+              tabletLocation.getTserverLocation(), tabletLocation.getTserverSession());
         } else {
           locToCache = tabletLocation;
         }
 
         // save endRow for next iteration
-        lastEndRow = locToCache.tablet_extent.endRow();
+        lastEndRow = locToCache.getExtent().endRow();
 
         updateCache(locToCache, lcSession);
       }
@@ -585,20 +586,20 @@
   }
 
   private void updateCache(TabletLocation tabletLocation, LockCheckerSession lcSession) {
-    if (!tabletLocation.tablet_extent.tableId().equals(tableId)) {
+    if (!tabletLocation.getExtent().tableId().equals(tableId)) {
       // sanity check
       throw new IllegalStateException(
-          "Unexpected extent returned " + tableId + "  " + tabletLocation.tablet_extent);
+          "Unexpected extent returned " + tableId + "  " + tabletLocation.getExtent());
     }
 
-    if (tabletLocation.tablet_location == null) {
+    if (tabletLocation.getTserverLocation() == null) {
       // sanity check
       throw new IllegalStateException(
-          "Cannot add null locations to cache " + tableId + "  " + tabletLocation.tablet_extent);
+          "Cannot add null locations to cache " + tableId + "  " + tabletLocation.getExtent());
     }
 
     // clear out any overlapping extents in cache
-    removeOverlapping(metaCache, tabletLocation.tablet_extent);
+    removeOverlapping(metaCache, tabletLocation.getExtent());
 
     // do not add to cache unless lock is held
     if (lcSession.checkLock(tabletLocation) == null) {
@@ -606,14 +607,14 @@
     }
 
     // add it to cache
-    Text er = tabletLocation.tablet_extent.endRow();
+    Text er = tabletLocation.getExtent().endRow();
     if (er == null) {
       er = MAX_TEXT;
     }
     metaCache.put(er, tabletLocation);
 
     if (!badExtents.isEmpty()) {
-      removeOverlapping(badExtents, tabletLocation.tablet_extent);
+      removeOverlapping(badExtents, tabletLocation.getExtent());
     }
   }
 
@@ -631,7 +632,7 @@
     while (iter.hasNext()) {
       Entry<Text,TabletLocation> entry = iter.next();
 
-      KeyExtent ke = entry.getValue().tablet_extent;
+      KeyExtent ke = entry.getValue().getExtent();
 
       if (stopRemoving(nke, ke)) {
         break;
@@ -663,7 +664,7 @@
     Entry<Text,TabletLocation> entry = metaCache.ceilingEntry(row);
 
     if (entry != null) {
-      KeyExtent ke = entry.getValue().tablet_extent;
+      KeyExtent ke = entry.getValue().getExtent();
       if (ke.prevEndRow() == null || ke.prevEndRow().compareTo(row) < 0) {
         return entry.getValue();
       }
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderIterator.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderIterator.java
index 322fd0f..b39240f 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderIterator.java
@@ -18,8 +18,6 @@
  */
 package org.apache.accumulo.core.clientImpl;
 
-import static java.util.concurrent.TimeUnit.MILLISECONDS;
-import static java.util.concurrent.TimeUnit.MINUTES;
 import static java.util.concurrent.TimeUnit.SECONDS;
 
 import java.io.IOException;
@@ -73,13 +71,12 @@
 import org.apache.accumulo.core.spi.scan.ScanServerAttempt;
 import org.apache.accumulo.core.spi.scan.ScanServerSelections;
 import org.apache.accumulo.core.spi.scan.ScanServerSelector;
+import org.apache.accumulo.core.tabletscan.thrift.ScanServerBusyException;
+import org.apache.accumulo.core.tabletscan.thrift.TSampleNotPresentException;
+import org.apache.accumulo.core.tabletscan.thrift.TabletScanClientService;
 import org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException;
-import org.apache.accumulo.core.tabletserver.thrift.ScanServerBusyException;
-import org.apache.accumulo.core.tabletserver.thrift.TSampleNotPresentException;
-import org.apache.accumulo.core.tabletserver.thrift.TabletScanClientService;
 import org.apache.accumulo.core.trace.TraceUtil;
 import org.apache.accumulo.core.util.ByteBufferUtil;
-import org.apache.accumulo.core.util.HostAndPort;
 import org.apache.accumulo.core.util.OpTimer;
 import org.apache.accumulo.core.util.Retry;
 import org.apache.thrift.TApplicationException;
@@ -88,6 +85,8 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.net.HostAndPort;
+
 public class TabletServerBatchReaderIterator implements Iterator<Entry<Key,Value>> {
 
   private static final Logger log = LoggerFactory.getLogger(TabletServerBatchReaderIterator.class);
@@ -160,17 +159,15 @@
           log.warn("Failed to add Batch Scan result", e);
         }
         fatalException = e;
-        throw new RuntimeException(e);
+        throw new IllegalStateException(e);
 
       }
     };
 
     try {
       lookup(ranges, rr);
-    } catch (RuntimeException re) {
-      throw re;
-    } catch (Exception e) {
-      throw new RuntimeException("Failed to create iterator", e);
+    } catch (AccumuloException | AccumuloSecurityException | TableNotFoundException e) {
+      throw new IllegalStateException("Failed to create iterator", e);
     }
   }
 
@@ -196,7 +193,7 @@
           if (fatalException instanceof RuntimeException) {
             throw (RuntimeException) fatalException;
           } else {
-            throw new RuntimeException(fatalException);
+            throw new IllegalStateException(fatalException);
           }
         }
 
@@ -207,13 +204,14 @@
               + " so that it can be closed when this Iterator is exhausted. Not"
               + " retaining a reference to the BatchScanner guarantees that you are"
               + " leaking threads in your client JVM.", shortMsg);
-          throw new RuntimeException(shortMsg + " Ensure proper handling of the BatchScanner.");
+          throw new IllegalStateException(
+              shortMsg + " Ensure proper handling of the BatchScanner.");
         }
 
         batchIterator = batch.iterator();
         return batch != LAST_BATCH;
       } catch (InterruptedException e) {
-        throw new RuntimeException(e);
+        throw new IllegalStateException(e);
       }
     }
   }
@@ -253,9 +251,9 @@
 
     int lastFailureSize = Integer.MAX_VALUE;
 
-    Retry retry = Retry.builder().infiniteRetries().retryAfter(100, MILLISECONDS)
-        .incrementBy(100, MILLISECONDS).maxWait(10, SECONDS).backOffFactor(1.07)
-        .logInterval(1, MINUTES).createFactory().createRetry();
+    Retry retry = Retry.builder().infiniteRetries().retryAfter(Duration.ofMillis(100))
+        .incrementBy(Duration.ofMillis(100)).maxWait(Duration.ofMinutes(10)).backOffFactor(1.07)
+        .logInterval(Duration.ofMinutes(1)).createFactory().createRetry();
 
     while (true) {
 
@@ -286,7 +284,7 @@
         try {
           retry.waitForNextAttempt(log, "binRanges retry failures");
         } catch (InterruptedException e) {
-          throw new RuntimeException(e);
+          throw new IllegalStateException(e);
         }
 
       }
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchWriter.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchWriter.java
index 980ba04..d54d144 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchWriter.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchWriter.java
@@ -19,8 +19,6 @@
 package org.apache.accumulo.core.clientImpl;
 
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
-import static java.util.concurrent.TimeUnit.MINUTES;
-import static java.util.concurrent.TimeUnit.SECONDS;
 import static java.util.function.Function.identity;
 import static java.util.stream.Collectors.toList;
 import static java.util.stream.Collectors.toMap;
@@ -29,6 +27,7 @@
 import java.lang.management.CompilationMXBean;
 import java.lang.management.GarbageCollectorMXBean;
 import java.lang.management.ManagementFactory;
+import java.time.Duration;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
@@ -61,6 +60,7 @@
 import org.apache.accumulo.core.client.TimedOutException;
 import org.apache.accumulo.core.clientImpl.TabletLocator.TabletServerMutations;
 import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.clientImpl.thrift.TInfo;
 import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.constraints.Violations;
 import org.apache.accumulo.core.data.ConstraintViolationSummary;
@@ -71,14 +71,12 @@
 import org.apache.accumulo.core.dataImpl.TabletIdImpl;
 import org.apache.accumulo.core.dataImpl.thrift.TMutation;
 import org.apache.accumulo.core.dataImpl.thrift.UpdateErrors;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock;
+import org.apache.accumulo.core.lock.ServiceLock;
 import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
+import org.apache.accumulo.core.tabletingest.thrift.TabletIngestClientService;
 import org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException;
-import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
 import org.apache.accumulo.core.trace.TraceUtil;
-import org.apache.accumulo.core.trace.thrift.TInfo;
-import org.apache.accumulo.core.util.HostAndPort;
 import org.apache.accumulo.core.util.Retry;
 import org.apache.accumulo.core.util.threads.ThreadPools;
 import org.apache.accumulo.core.util.threads.Threads;
@@ -90,6 +88,7 @@
 import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Joiner;
+import com.google.common.net.HostAndPort;
 
 import io.opentelemetry.api.trace.Span;
 import io.opentelemetry.context.Scope;
@@ -506,7 +505,7 @@
         wait();
       }
     } catch (InterruptedException e) {
-      throw new RuntimeException(e);
+      throw new IllegalStateException(e);
     }
   }
 
@@ -949,13 +948,13 @@
       // happen after the batch writer closes. See #3721
       try {
         final HostAndPort parsedServer = HostAndPort.fromString(location);
-        final TabletClientService.Iface client;
+        final TabletIngestClientService.Iface client;
 
         if (timeoutTracker.getTimeOut() < context.getClientTimeoutInMillis()) {
-          client = ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, parsedServer, context,
+          client = ThriftUtil.getClient(ThriftClientTypes.TABLET_INGEST, parsedServer, context,
               timeoutTracker.getTimeOut());
         } else {
-          client = ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, parsedServer, context);
+          client = ThriftUtil.getClient(ThriftClientTypes.TABLET_INGEST, parsedServer, context);
         }
 
         try {
@@ -1093,9 +1092,9 @@
 
       private void cancelSession() throws InterruptedException, ThriftSecurityException {
 
-        Retry retry = Retry.builder().infiniteRetries().retryAfter(100, MILLISECONDS)
-            .incrementBy(100, MILLISECONDS).maxWait(60, SECONDS).backOffFactor(1.5)
-            .logInterval(3, MINUTES).createRetry();
+        Retry retry = Retry.builder().infiniteRetries().retryAfter(Duration.ofMillis(100))
+            .incrementBy(Duration.ofMillis(100)).maxWait(Duration.ofMinutes(1)).backOffFactor(1.5)
+            .logInterval(Duration.ofMinutes(3)).createRetry();
 
         final HostAndPort parsedServer = HostAndPort.fromString(location);
 
@@ -1108,7 +1107,7 @@
         // exceptions.
         while (!somethingFailed.get()) {
 
-          TabletClientService.Client client = null;
+          TabletIngestClientService.Client client = null;
 
           // Check if a lock is held by any tserver at the host and port. It does not need to be the
           // exact tserver instance that existed when the session was created because if a new
@@ -1123,10 +1122,10 @@
 
           try {
             if (timeout < context.getClientTimeoutInMillis()) {
-              client = ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, parsedServer, context,
+              client = ThriftUtil.getClient(ThriftClientTypes.TABLET_INGEST, parsedServer, context,
                   timeout);
             } else {
-              client = ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, parsedServer, context);
+              client = ThriftUtil.getClient(ThriftClientTypes.TABLET_INGEST, parsedServer, context);
             }
 
             if (useCloseUpdate) {
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftScanner.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftScanner.java
index c76277f..1402732 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftScanner.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftScanner.java
@@ -19,9 +19,9 @@
 package org.apache.accumulo.core.clientImpl;
 
 import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 
 import java.io.IOException;
-import java.security.SecureRandom;
 import java.time.Duration;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -34,7 +34,6 @@
 import java.util.Set;
 import java.util.SortedMap;
 import java.util.SortedSet;
-import java.util.concurrent.TimeUnit;
 import java.util.function.Supplier;
 import java.util.stream.Collectors;
 
@@ -46,6 +45,7 @@
 import org.apache.accumulo.core.client.TimedOutException;
 import org.apache.accumulo.core.client.sample.SamplerConfiguration;
 import org.apache.accumulo.core.clientImpl.TabletLocator.TabletLocation;
+import org.apache.accumulo.core.clientImpl.thrift.TInfo;
 import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Column;
@@ -69,15 +69,13 @@
 import org.apache.accumulo.core.spi.scan.ScanServerAttempt;
 import org.apache.accumulo.core.spi.scan.ScanServerSelections;
 import org.apache.accumulo.core.spi.scan.ScanServerSelector;
+import org.apache.accumulo.core.tabletscan.thrift.ScanServerBusyException;
+import org.apache.accumulo.core.tabletscan.thrift.TSampleNotPresentException;
+import org.apache.accumulo.core.tabletscan.thrift.TabletScanClientService;
+import org.apache.accumulo.core.tabletscan.thrift.TooManyFilesException;
 import org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException;
 import org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException;
-import org.apache.accumulo.core.tabletserver.thrift.ScanServerBusyException;
-import org.apache.accumulo.core.tabletserver.thrift.TSampleNotPresentException;
-import org.apache.accumulo.core.tabletserver.thrift.TabletScanClientService;
-import org.apache.accumulo.core.tabletserver.thrift.TooManyFilesException;
 import org.apache.accumulo.core.trace.TraceUtil;
-import org.apache.accumulo.core.trace.thrift.TInfo;
-import org.apache.accumulo.core.util.HostAndPort;
 import org.apache.accumulo.core.util.OpTimer;
 import org.apache.accumulo.core.util.Retry;
 import org.apache.hadoop.io.Text;
@@ -86,6 +84,8 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.net.HostAndPort;
+
 import io.opentelemetry.api.trace.Span;
 import io.opentelemetry.context.Scope;
 
@@ -105,7 +105,6 @@
   // metadata
   public static final Map<TabletType,Set<String>> serversWaitedForWrites =
       new EnumMap<>(TabletType.class);
-  private static final SecureRandom random = new SecureRandom();
 
   static {
     for (TabletType ttype : TabletType.values()) {
@@ -270,9 +269,9 @@
   static <T> Optional<T> waitUntil(Supplier<Optional<T>> condition, Duration maxWaitTime,
       String description, Duration timeoutLeft, ClientContext context, TableId tableId,
       Logger log) {
-    Retry retry = Retry.builder().infiniteRetries().retryAfter(100, TimeUnit.MILLISECONDS)
-        .incrementBy(100, TimeUnit.MILLISECONDS).maxWait(1, SECONDS).backOffFactor(1.5)
-        .logInterval(3, TimeUnit.MINUTES).createRetry();
+    Retry retry = Retry.builder().infiniteRetries().retryAfter(Duration.ofMillis(100))
+        .incrementBy(Duration.ofMillis(100)).maxWait(Duration.ofSeconds(1)).backOffFactor(1.5)
+        .logInterval(Duration.ofMinutes(3)).createRetry();
 
     long startTime = System.nanoTime();
     Optional<T> optional = condition.get();
@@ -319,7 +318,7 @@
       Thread.sleep(millis);
     }
     // wait 2 * last time, with +-10% random jitter
-    return (long) (Math.min(millis * 2, maxSleep) * (.9 + random.nextDouble() / 5));
+    return (long) (Math.min(millis * 2, maxSleep) * (.9 + RANDOM.get().nextDouble() / 5));
   }
 
   public static List<KeyValue> scan(ClientContext context, ScanState scanState, long timeOut)
@@ -374,18 +373,18 @@
             } else {
               // when a tablet splits we do want to continue scanning the low child
               // of the split if we are already passed it
-              Range dataRange = loc.tablet_extent.toDataRange();
+              Range dataRange = loc.getExtent().toDataRange();
 
               if (scanState.range.getStartKey() != null
                   && dataRange.afterEndKey(scanState.range.getStartKey())) {
                 // go to the next tablet
-                scanState.startRow = loc.tablet_extent.endRow();
+                scanState.startRow = loc.getExtent().endRow();
                 scanState.skipStartRow = true;
                 loc = null;
               } else if (scanState.range.getEndKey() != null
                   && dataRange.beforeStartKey(scanState.range.getEndKey())) {
                 // should not happen
-                throw new RuntimeException("Unexpected tablet, extent : " + loc.tablet_extent
+                throw new IllegalStateException("Unexpected tablet, extent : " + loc.getExtent()
                     + "  range : " + scanState.range + " startRow : " + scanState.startRow);
               }
             }
@@ -411,7 +410,7 @@
         }
 
         Span child2 = TraceUtil.startSpan(ThriftScanner.class, "scan::location",
-            Map.of("tserver", loc.tablet_location));
+            Map.of("tserver", loc.getTserverLocation()));
         try (Scope scanLocation = child2.makeCurrent()) {
           results = scan(loc, scanState, context, timeOut, startTime);
         } catch (AccumuloSecurityException e) {
@@ -422,7 +421,7 @@
           throw e;
         } catch (TApplicationException tae) {
           TraceUtil.setException(child2, tae, true);
-          throw new AccumuloServerException(scanState.getErrorLocation().tablet_location, tae);
+          throw new AccumuloServerException(scanState.getErrorLocation().getTserverLocation(), tae);
         } catch (TSampleNotPresentException tsnpe) {
           String message = "Table " + context.getPrintableTableInfoFromId(scanState.tableId)
               + " does not have sampling configured or built";
@@ -437,7 +436,7 @@
           }
           lastError = error;
 
-          TabletLocator.getLocator(context, scanState.tableId).invalidateCache(loc.tablet_extent);
+          TabletLocator.getLocator(context, scanState.tableId).invalidateCache(loc.getExtent());
           loc = null;
 
           // no need to try the current scan id somewhere else
@@ -512,7 +511,7 @@
           sleepMillis = pause(sleepMillis, maxSleepTime, scanState.runOnScanServer);
         } catch (TException e) {
           TabletLocator.getLocator(context, scanState.tableId).invalidateCache(context,
-              loc.tablet_location);
+              loc.getTserverLocation());
           error = "Scan failed, thrift error " + e.getClass().getName() + "  " + e.getMessage()
               + " " + scanState.getErrorLocation();
           if (!error.equals(lastError)) {
@@ -564,17 +563,17 @@
 
       TabletLocation newLoc;
 
-      var tabletId = new TabletIdImpl(loc.tablet_extent);
+      var tabletId = new TabletIdImpl(loc.getExtent());
 
       if (scanState.scanID != null && scanState.prevLoc != null
-          && scanState.prevLoc.tablet_session.equals("scan_server")
-          && scanState.prevLoc.tablet_extent.equals(loc.tablet_extent)) {
+          && scanState.prevLoc.getTserverSession().equals("scan_server")
+          && scanState.prevLoc.getExtent().equals(loc.getExtent())) {
         // this is the case of continuing a scan on a scan server for the same tablet, so lets not
         // call the scan server selector and just go back to the previous scan server
         newLoc = scanState.prevLoc;
         log.trace(
             "For tablet {} continuing scan on scan server {} without consulting scan server selector, using busyTimeout {}",
-            loc.tablet_extent, newLoc.tablet_location, scanState.busyTimeout);
+            loc.getExtent(), newLoc.getTserverLocation(), scanState.busyTimeout);
       } else {
         // obtain a snapshot once and only expose this snapshot to the plugin for consistency
         var attempts = scanState.scanAttempts.snapshot();
@@ -606,7 +605,7 @@
           public <T> Optional<T> waitUntil(Supplier<Optional<T>> condition, Duration maxWaitTime,
               String description) {
             return ThriftScanner.waitUntil(condition, maxWaitTime, description, timeoutLeft,
-                context, loc.tablet_extent.tableId(), log);
+                context, loc.getExtent().tableId(), log);
           }
         };
 
@@ -616,19 +615,18 @@
 
         String scanServer = actions.getScanServer(tabletId);
         if (scanServer != null) {
-          newLoc = new TabletLocation(loc.tablet_extent, scanServer, "scan_server");
+          newLoc = new TabletLocation(loc.getExtent(), scanServer, "scan_server");
           delay = actions.getDelay();
           scanState.busyTimeout = actions.getBusyTimeout();
           log.trace(
               "For tablet {} using hints {} scan server selector chose scan_server:{} delay:{} busyTimeout:{}",
-              loc.tablet_extent, scanState.executionHints, scanServer, delay,
-              scanState.busyTimeout);
+              loc.getExtent(), scanState.executionHints, scanServer, delay, scanState.busyTimeout);
         } else {
           newLoc = loc;
           delay = actions.getDelay();
           scanState.busyTimeout = Duration.ZERO;
           log.trace("For tablet {} using hints {} scan server selector chose tablet_server",
-              loc.tablet_extent, scanState.executionHints);
+              loc.getExtent(), scanState.executionHints);
         }
 
         if (!delay.isZero()) {
@@ -641,7 +639,7 @@
         }
       }
 
-      var reporter = scanState.scanAttempts.createReporter(newLoc.tablet_location, tabletId);
+      var reporter = scanState.scanAttempts.createReporter(newLoc.getTserverLocation(), tabletId);
 
       try {
         return scanRpc(newLoc, scanState, context, scanState.busyTimeout.toMillis());
@@ -666,7 +664,7 @@
 
     final TInfo tinfo = TraceUtil.traceInfo();
 
-    final HostAndPort parsedLocation = HostAndPort.fromString(loc.tablet_location);
+    final HostAndPort parsedLocation = HostAndPort.fromString(loc.getTserverLocation());
     TabletScanClientService.Client client =
         ThriftUtil.getClient(ThriftClientTypes.TABLET_SCAN, parsedLocation, context);
 
@@ -681,23 +679,24 @@
       scanState.prevLoc = loc;
 
       if (scanState.scanID == null) {
-        Thread.currentThread().setName("Starting scan tserver=" + loc.tablet_location + " tableId="
-            + loc.tablet_extent.tableId());
+        Thread.currentThread().setName("Starting scan tserver=" + loc.getTserverLocation()
+            + " tableId=" + loc.getExtent().tableId());
 
         if (log.isTraceEnabled()) {
-          String msg = "Starting scan tserver=" + loc.tablet_location + " tablet="
-              + loc.tablet_extent + " range=" + scanState.range + " ssil="
+          String msg = "Starting scan tserver=" + loc.getTserverLocation() + " tablet="
+              + loc.getExtent() + " range=" + scanState.range + " ssil="
               + scanState.serverSideIteratorList + " ssio=" + scanState.serverSideIteratorOptions
               + " context=" + scanState.classLoaderContext;
           log.trace("tid={} {}", Thread.currentThread().getId(), msg);
           timer = new OpTimer().start();
         }
 
-        TabletType ttype = TabletType.type(loc.tablet_extent);
-        boolean waitForWrites = !serversWaitedForWrites.get(ttype).contains(loc.tablet_location);
+        TabletType ttype = TabletType.type(loc.getExtent());
+        boolean waitForWrites =
+            !serversWaitedForWrites.get(ttype).contains(loc.getTserverLocation());
 
         InitialScan is = client.startScan(tinfo, scanState.context.rpcCreds(),
-            loc.tablet_extent.toThrift(), scanState.range.toThrift(),
+            loc.getExtent().toThrift(), scanState.range.toThrift(),
             scanState.columns.stream().map(Column::toThrift).collect(Collectors.toList()),
             scanState.size, scanState.serverSideIteratorList, scanState.serverSideIteratorOptions,
             scanState.authorizations.getAuthorizationsBB(), waitForWrites, scanState.isolated,
@@ -705,7 +704,7 @@
             SamplerConfigurationImpl.toThrift(scanState.samplerConfig), scanState.batchTimeOut,
             scanState.classLoaderContext, scanState.executionHints, busyTimeout);
         if (waitForWrites) {
-          serversWaitedForWrites.get(ttype).add(loc.tablet_location);
+          serversWaitedForWrites.get(ttype).add(loc.getTserverLocation());
         }
 
         sr = is.result;
@@ -719,7 +718,7 @@
       } else {
         // log.debug("Calling continue scan : "+scanState.range+" loc = "+loc);
         String msg =
-            "Continuing scan tserver=" + loc.tablet_location + " scanid=" + scanState.scanID;
+            "Continuing scan tserver=" + loc.getTserverLocation() + " scanid=" + scanState.scanID;
         Thread.currentThread().setName(msg);
 
         if (log.isTraceEnabled()) {
@@ -744,7 +743,7 @@
       } else {
         // log.debug("No more : tab end row = "+loc.tablet_extent.getEndRow()+" range =
         // "+scanState.range);
-        if (loc.tablet_extent.endRow() == null) {
+        if (loc.getExtent().endRow() == null) {
           scanState.finished = true;
 
           if (timer != null) {
@@ -755,8 +754,8 @@
           }
 
         } else if (scanState.range.getEndKey() == null || !scanState.range
-            .afterEndKey(new Key(loc.tablet_extent.endRow()).followingKey(PartialKey.ROW))) {
-          scanState.startRow = loc.tablet_extent.endRow();
+            .afterEndKey(new Key(loc.getExtent().endRow()).followingKey(PartialKey.ROW))) {
+          scanState.startRow = loc.getExtent().endRow();
           scanState.skipStartRow = true;
 
           if (timer != null) {
@@ -803,7 +802,7 @@
       TInfo tinfo = TraceUtil.traceInfo();
 
       log.debug("Closing active scan {} {}", scanState.prevLoc, scanState.scanID);
-      HostAndPort parsedLocation = HostAndPort.fromString(scanState.prevLoc.tablet_location);
+      HostAndPort parsedLocation = HostAndPort.fromString(scanState.prevLoc.getTserverLocation());
       TabletScanClientService.Client client = null;
       try {
         client =
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftTransportKey.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftTransportKey.java
index f4c7047..0f84154 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftTransportKey.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftTransportKey.java
@@ -25,9 +25,9 @@
 import org.apache.accumulo.core.rpc.SaslConnectionParams;
 import org.apache.accumulo.core.rpc.SslConnectionParams;
 import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
-import org.apache.accumulo.core.util.HostAndPort;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.net.HostAndPort;
 
 @VisibleForTesting
 public class ThriftTransportKey {
@@ -58,7 +58,7 @@
     this.saslParams = saslParams;
     if (saslParams != null && sslParams != null) {
       // TSasl and TSSL transport factories don't play nicely together
-      throw new RuntimeException("Cannot use both SSL and SASL thrift transports");
+      throw new IllegalArgumentException("Cannot use both SSL and SASL thrift transports");
     }
     this.hash = Objects.hash(type, server, timeout, sslParams, saslParams);
   }
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftTransportPool.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftTransportPool.java
index 6f62125..1863cf6 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftTransportPool.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftTransportPool.java
@@ -20,8 +20,8 @@
 
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static java.util.concurrent.TimeUnit.MINUTES;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 
-import java.security.SecureRandom;
 import java.util.ArrayDeque;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -43,7 +43,6 @@
 
 import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
-import org.apache.accumulo.core.util.HostAndPort;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.core.util.threads.Threads;
 import org.apache.thrift.TConfiguration;
@@ -53,13 +52,13 @@
 import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Preconditions;
+import com.google.common.net.HostAndPort;
 
 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 
 public class ThriftTransportPool {
 
   private static final Logger log = LoggerFactory.getLogger(ThriftTransportPool.class);
-  private static final SecureRandom random = new SecureRandom();
   private static final long ERROR_THRESHOLD = 20L;
   private static final long STUCK_THRESHOLD = MINUTES.toMillis(2);
 
@@ -134,7 +133,7 @@
     if (serversSet.isEmpty()) {
       return null;
     }
-    Collections.shuffle(serversSet, random);
+    Collections.shuffle(serversSet, RANDOM.get());
     for (ThriftTransportKey ttk : serversSet) {
       CachedConnection connection = connectionPool.reserveAny(ttk);
       if (connection != null) {
@@ -244,7 +243,7 @@
     try {
       checkThread.join();
     } catch (InterruptedException e) {
-      throw new RuntimeException(e);
+      throw new IllegalStateException(e);
     }
   }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/UserCompactionUtils.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/UserCompactionUtils.java
index 2c71bc5..e64ae66 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/UserCompactionUtils.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/UserCompactionUtils.java
@@ -217,8 +217,6 @@
         is.write(dout);
       }
 
-      CompactionStrategyConfigUtil.encode(dout, cc);
-
       encodeConfigurer(dout, cc.getConfigurer());
       encodeSelector(dout, cc.getSelector());
       encode(dout, cc.getExecutionHints());
@@ -260,8 +258,6 @@
 
       cc.setIterators(iterators);
 
-      CompactionStrategyConfigUtil.decode(cc, din);
-
       var configurer = decodeConfigurer(din);
       if (!isDefault(configurer)) {
         cc.setConfigurer(configurer);
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/Writer.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/Writer.java
deleted file mode 100644
index 9f124ed..0000000
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/Writer.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.clientImpl;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.metadata.schema.Ample;
-
-public class Writer {
-
-  private ClientContext context;
-  private TableId tableId;
-
-  public Writer(ClientContext context, TableId tableId) {
-    checkArgument(context != null, "context is null");
-    checkArgument(tableId != null, "tableId is null");
-    this.context = context;
-    this.tableId = tableId;
-  }
-
-  public void update(Mutation m) throws AccumuloException, TableNotFoundException {
-    checkArgument(m != null, "m is null");
-
-    if (m.size() == 0) {
-      throw new IllegalArgumentException("Can not add empty mutations");
-    }
-
-    String table = Ample.DataLevel.of(tableId).metaTable();
-
-    try (var writer = context.createBatchWriter(table)) {
-      writer.addMutation(m);
-    }
-
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ZookeeperLockChecker.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ZookeeperLockChecker.java
index 23f036b..39c7cec 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ZookeeperLockChecker.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ZookeeperLockChecker.java
@@ -20,8 +20,8 @@
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.clientImpl.TabletLocatorImpl.TabletServerLockChecker;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock;
 import org.apache.accumulo.core.fate.zookeeper.ZooCache;
+import org.apache.accumulo.core.lock.ServiceLock;
 
 public class ZookeeperLockChecker implements TabletServerLockChecker {
 
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/BulkImport.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/BulkImport.java
index f13420d..8689a56 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/BulkImport.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/BulkImport.java
@@ -19,8 +19,6 @@
 package org.apache.accumulo.core.clientImpl.bulk;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
-import static java.util.concurrent.TimeUnit.MILLISECONDS;
-import static java.util.concurrent.TimeUnit.MINUTES;
 import static java.util.stream.Collectors.groupingBy;
 import static org.apache.accumulo.core.file.blockfile.impl.CachableBlockFile.pathToCacheId;
 import static org.apache.accumulo.core.util.Validators.EXISTING_TABLE_NAME;
@@ -28,6 +26,7 @@
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.time.Duration;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -74,6 +73,7 @@
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.file.FileOperations;
 import org.apache.accumulo.core.file.FileSKVIterator;
+import org.apache.accumulo.core.metadata.UnreferencedTabletFile;
 import org.apache.accumulo.core.spi.crypto.CryptoService;
 import org.apache.accumulo.core.util.Retry;
 import org.apache.accumulo.core.volume.VolumeConfiguration;
@@ -85,9 +85,9 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
 import com.google.common.base.Preconditions;
-import com.google.common.cache.Cache;
-import com.google.common.cache.CacheBuilder;
 import com.google.common.collect.Sets;
 
 public class BulkImport implements ImportDestinationArguments, ImportMappingOptions {
@@ -141,9 +141,9 @@
     if (propValue != null) {
       maxTablets = Integer.parseInt(propValue);
     }
-    Retry retry = Retry.builder().infiniteRetries().retryAfter(100, MILLISECONDS)
-        .incrementBy(100, MILLISECONDS).maxWait(2, MINUTES).backOffFactor(1.5)
-        .logInterval(3, MINUTES).createRetry();
+    Retry retry = Retry.builder().infiniteRetries().retryAfter(Duration.ofMillis(100))
+        .incrementBy(Duration.ofMillis(100)).maxWait(Duration.ofMinutes(2)).backOffFactor(1.5)
+        .logInterval(Duration.ofMinutes(3)).createRetry();
 
     // retry if a merge occurs
     boolean shouldRetry = true;
@@ -179,7 +179,7 @@
         try {
           retry.waitForNextAttempt(log, String.format("bulk import to %s(%s)", tableName, tableId));
         } catch (InterruptedException e) {
-          throw new RuntimeException(e);
+          throw new IllegalStateException(e);
         }
         log.info(ae.getMessage() + ". Retrying bulk import to " + tableName);
       }
@@ -261,9 +261,9 @@
     long l;
   }
 
-  public static Map<KeyExtent,Long> estimateSizes(AccumuloConfiguration acuConf, Path mapFile,
-      long fileSize, Collection<KeyExtent> extents, FileSystem ns, Cache<String,Long> fileLenCache,
-      CryptoService cs) throws IOException {
+  public static Map<KeyExtent,Long> estimateSizes(AccumuloConfiguration acuConf,
+      UnreferencedTabletFile dataFile, long fileSize, Collection<KeyExtent> extents, FileSystem ns,
+      Cache<String,Long> fileLenCache, CryptoService cs) throws IOException {
 
     if (extents.size() == 1) {
       return Collections.singletonMap(extents.iterator().next(), fileSize);
@@ -277,17 +277,14 @@
 
     Text row = new Text();
 
-    FileSKVIterator index = FileOperations.getInstance().newIndexReaderBuilder()
-        .forFile(mapFile.toString(), ns, ns.getConf(), cs).withTableConfiguration(acuConf)
-        .withFileLenCache(fileLenCache).build();
-
-    try {
+    try (FileSKVIterator index =
+        FileOperations.getInstance().newIndexReaderBuilder().forFile(dataFile, ns, ns.getConf(), cs)
+            .withTableConfiguration(acuConf).withFileLenCache(fileLenCache).build()) {
       while (index.hasTop()) {
         Key key = index.getTopKey();
         totalIndexEntries++;
         key.getRow(row);
 
-        // TODO this could use a binary search
         for (Entry<KeyExtent,MLong> entry : counts.entrySet()) {
           if (entry.getKey().contains(row)) {
             entry.getValue().l++;
@@ -296,14 +293,6 @@
 
         index.next();
       }
-    } finally {
-      try {
-        if (index != null) {
-          index.close();
-        }
-      } catch (IOException e) {
-        log.debug("Failed to close " + mapFile, e);
-      }
     }
 
     Map<KeyExtent,Long> results = new TreeMap<>();
@@ -354,12 +343,11 @@
   }
 
   public static List<KeyExtent> findOverlappingTablets(ClientContext context,
-      KeyExtentCache extentCache, Path file, FileSystem fs, Cache<String,Long> fileLenCache,
-      CryptoService cs) throws IOException {
+      KeyExtentCache extentCache, UnreferencedTabletFile file, FileSystem fs,
+      Cache<String,Long> fileLenCache, CryptoService cs) throws IOException {
     try (FileSKVIterator reader = FileOperations.getInstance().newReaderBuilder()
-        .forFile(file.toString(), fs, fs.getConf(), cs)
-        .withTableConfiguration(context.getConfiguration()).withFileLenCache(fileLenCache)
-        .seekToBeginning().build()) {
+        .forFile(file, fs, fs.getConf(), cs).withTableConfiguration(context.getConfiguration())
+        .withFileLenCache(fileLenCache).seekToBeginning().build()) {
       return findOverlappingTablets(extentCache, reader);
     }
   }
@@ -379,7 +367,7 @@
     Map<String,Long> absFileLens = new HashMap<>();
     fileLens.forEach((k, v) -> absFileLens.put(pathToCacheId(new Path(dir, k)), v));
 
-    Cache<String,Long> fileLenCache = CacheBuilder.newBuilder().build();
+    Cache<String,Long> fileLenCache = Caffeine.newBuilder().build();
 
     fileLenCache.putAll(absFileLens);
 
@@ -411,13 +399,7 @@
     fileDestinations.values().stream().flatMap(List::stream)
         .filter(dest -> dest.getRangeType() == RangeType.FILE)
         .flatMap(dest -> Stream.of(dest.getStartRow(), dest.getEndRow())).filter(Objects::nonNull)
-        .map(Text::new).sorted().distinct().forEach(row -> {
-          try {
-            extentCache.lookup(row);
-          } catch (Exception e) {
-            throw new RuntimeException(e);
-          }
-        });
+        .map(Text::new).sorted().distinct().forEach(extentCache::lookup);
 
     SortedMap<KeyExtent,Files> mapping = new TreeMap<>();
 
@@ -548,22 +530,22 @@
         context.instanceOperations().getSystemConfiguration(), tableProps, tableId);
 
     for (FileStatus fileStatus : files) {
-      Path filePath = fileStatus.getPath();
+      UnreferencedTabletFile file = UnreferencedTabletFile.of(fs, fileStatus.getPath());
       CompletableFuture<Map<KeyExtent,Bulk.FileInfo>> future = CompletableFuture.supplyAsync(() -> {
         try {
           long t1 = System.currentTimeMillis();
           List<KeyExtent> extents =
-              findOverlappingTablets(context, extentCache, filePath, fs, fileLensCache, cs);
+              findOverlappingTablets(context, extentCache, file, fs, fileLensCache, cs);
           // make sure file isn't going to too many tablets
-          checkTabletCount(maxTablets, extents.size(), filePath.toString());
-          Map<KeyExtent,Long> estSizes = estimateSizes(context.getConfiguration(), filePath,
+          checkTabletCount(maxTablets, extents.size(), file.toString());
+          Map<KeyExtent,Long> estSizes = estimateSizes(context.getConfiguration(), file,
               fileStatus.getLen(), extents, fs, fileLensCache, cs);
           Map<KeyExtent,Bulk.FileInfo> pathLocations = new HashMap<>();
           for (KeyExtent ke : extents) {
-            pathLocations.put(ke, new Bulk.FileInfo(filePath, estSizes.getOrDefault(ke, 0L)));
+            pathLocations.put(ke, new Bulk.FileInfo(file.getPath(), estSizes.getOrDefault(ke, 0L)));
           }
           long t2 = System.currentTimeMillis();
-          log.debug("Mapped {} to {} tablets in {}ms", filePath, pathLocations.size(), t2 - t1);
+          log.debug("Mapped {} to {} tablets in {}ms", file, pathLocations.size(), t2 - t1);
           return pathLocations;
         } catch (Exception e) {
           throw new CompletionException(e);
@@ -581,9 +563,9 @@
         pathMapping.forEach((ext, fi) -> mappings.computeIfAbsent(ext, k -> new Files()).add(fi));
       } catch (InterruptedException e) {
         Thread.currentThread().interrupt();
-        throw new RuntimeException(e);
+        throw new IllegalStateException(e);
       } catch (ExecutionException e) {
-        throw new RuntimeException(e);
+        throw new IllegalStateException(e);
       }
     }
 
@@ -606,7 +588,7 @@
         if (ke.contains(oke)) {
           mappings.get(ke).merge(mappings.remove(oke));
         } else if (!oke.contains(ke)) {
-          throw new RuntimeException("Error during bulk import: Unable to merge overlapping "
+          throw new IllegalStateException("Error during bulk import: Unable to merge overlapping "
               + "tablets where neither tablet contains the other. This may be caused by "
               + "a concurrent merge. Key extents " + oke + " and " + ke + " overlap, but "
               + "neither contains the other.");
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/BulkSerialize.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/BulkSerialize.java
index 0f8d46c..533a43c 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/BulkSerialize.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/BulkSerialize.java
@@ -19,6 +19,7 @@
 package org.apache.accumulo.core.clientImpl.bulk;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.core.util.LazySingletons.GSON;
 
 import java.io.BufferedReader;
 import java.io.BufferedWriter;
@@ -27,10 +28,6 @@
 import java.io.InputStreamReader;
 import java.io.OutputStream;
 import java.io.OutputStreamWriter;
-import java.lang.reflect.Type;
-import java.util.Base64;
-import java.util.Base64.Decoder;
-import java.util.Base64.Encoder;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
@@ -41,17 +38,10 @@
 import org.apache.accumulo.core.clientImpl.bulk.Bulk.Mapping;
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.util.json.ByteArrayToBase64TypeAdapter;
 import org.apache.hadoop.fs.Path;
 
 import com.google.gson.Gson;
-import com.google.gson.GsonBuilder;
-import com.google.gson.JsonDeserializationContext;
-import com.google.gson.JsonDeserializer;
-import com.google.gson.JsonElement;
-import com.google.gson.JsonParseException;
-import com.google.gson.JsonPrimitive;
-import com.google.gson.JsonSerializationContext;
-import com.google.gson.JsonSerializer;
 import com.google.gson.reflect.TypeToken;
 import com.google.gson.stream.JsonWriter;
 
@@ -60,28 +50,7 @@
  */
 public class BulkSerialize {
 
-  private static class ByteArrayToBase64TypeAdapter
-      implements JsonSerializer<byte[]>, JsonDeserializer<byte[]> {
-
-    Decoder decoder = Base64.getUrlDecoder();
-    Encoder encoder = Base64.getUrlEncoder();
-
-    @Override
-    public byte[] deserialize(JsonElement json, Type typeOfT, JsonDeserializationContext context)
-        throws JsonParseException {
-      return decoder.decode(json.getAsString());
-    }
-
-    @Override
-    public JsonElement serialize(byte[] src, Type typeOfSrc, JsonSerializationContext context) {
-      return new JsonPrimitive(encoder.encodeToString(src));
-    }
-  }
-
-  static Gson createGson() {
-    return new GsonBuilder()
-        .registerTypeHierarchyAdapter(byte[].class, new ByteArrayToBase64TypeAdapter()).create();
-  }
+  private static final Gson gson = ByteArrayToBase64TypeAdapter.createBase64Gson();
 
   public interface Output {
     OutputStream create(Path path) throws IOException;
@@ -100,7 +69,6 @@
 
     try (OutputStream fsOut = output.create(lmFile); JsonWriter writer =
         new JsonWriter(new BufferedWriter(new OutputStreamWriter(fsOut, UTF_8)))) {
-      Gson gson = createGson();
       writer.setIndent("  ");
       writer.beginArray();
       Set<Entry<KeyExtent,Files>> es = loadMapping.entrySet();
@@ -130,7 +98,7 @@
     final Path renamingFile = new Path(bulkDir, Constants.BULK_RENAME_FILE);
     try (OutputStream fsOut = output.create(renamingFile);
         BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(fsOut))) {
-      new Gson().toJson(oldToNewNameMap, writer);
+      GSON.get().toJson(oldToNewNameMap, writer);
     }
   }
 
@@ -141,7 +109,6 @@
   public static Map<String,String> readRenameMap(String bulkDir, Input input) throws IOException {
     final Path renamingFile = new Path(bulkDir, Constants.BULK_RENAME_FILE);
     Map<String,String> oldToNewNameMap;
-    Gson gson = createGson();
     try (InputStream fis = input.open(renamingFile);
         BufferedReader reader = new BufferedReader(new InputStreamReader(fis))) {
       oldToNewNameMap = gson.fromJson(reader, new TypeToken<Map<String,String>>() {}.getType());
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/ConcurrentKeyExtentCache.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/ConcurrentKeyExtentCache.java
index f17639a..e338fe9 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/ConcurrentKeyExtentCache.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/ConcurrentKeyExtentCache.java
@@ -23,7 +23,6 @@
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashSet;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map.Entry;
 import java.util.Objects;
@@ -88,9 +87,9 @@
 
   @VisibleForTesting
   protected Stream<KeyExtent> lookupExtents(Text row) {
-    return TabletsMetadata.builder(ctx).forTable(tableId).overlapping(row, true, null)
-        .checkConsistency().fetch(PREV_ROW).build().stream().limit(100)
-        .map(TabletMetadata::getExtent);
+    TabletsMetadata tabletsMetadata = TabletsMetadata.builder(ctx).forTable(tableId)
+        .overlapping(row, true, null).checkConsistency().fetch(PREV_ROW).build();
+    return tabletsMetadata.stream().limit(100).map(TabletMetadata::getExtent);
   }
 
   @Override
@@ -129,15 +128,8 @@
         for (Text lookupRow : lookupRows) {
           if (getFromCache(lookupRow) == null) {
             while (true) {
-              try {
-                Iterator<KeyExtent> iter = lookupExtents(lookupRow).iterator();
-                while (iter.hasNext()) {
-                  KeyExtent ke2 = iter.next();
-                  if (inCache(ke2)) {
-                    break;
-                  }
-                  updateCache(ke2);
-                }
+              try (Stream<KeyExtent> keyExtentStream = lookupExtents(lookupRow)) {
+                keyExtentStream.takeWhile(ke2 -> !inCache(ke2)).forEach(this::updateCache);
                 break;
               } catch (TabletDeletedException tde) {
                 // tablets were merged away in the table, start over and try again
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/LoadMappingIterator.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/LoadMappingIterator.java
index 30e1041..36b2d5b 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/LoadMappingIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/LoadMappingIterator.java
@@ -19,7 +19,6 @@
 package org.apache.accumulo.core.clientImpl.bulk;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
-import static org.apache.accumulo.core.clientImpl.bulk.BulkSerialize.createGson;
 
 import java.io.BufferedReader;
 import java.io.IOException;
@@ -32,6 +31,7 @@
 
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.util.json.ByteArrayToBase64TypeAdapter;
 
 import com.google.gson.Gson;
 import com.google.gson.stream.JsonReader;
@@ -41,14 +41,14 @@
  */
 public class LoadMappingIterator
     implements Iterator<Map.Entry<KeyExtent,Bulk.Files>>, AutoCloseable {
-  private TableId tableId;
-  private JsonReader reader;
-  private Gson gson = createGson();
+  private final TableId tableId;
+  private final JsonReader reader;
+  private static final Gson gson = ByteArrayToBase64TypeAdapter.createBase64Gson();
   private Map<String,String> renameMap;
 
-  LoadMappingIterator(TableId tableId, InputStream loadMapFile) throws IOException {
+  LoadMappingIterator(TableId tableId, InputStream loadMappingFile) throws IOException {
     this.tableId = tableId;
-    this.reader = new JsonReader(new BufferedReader(new InputStreamReader(loadMapFile, UTF_8)));
+    this.reader = new JsonReader(new BufferedReader(new InputStreamReader(loadMappingFile, UTF_8)));
     this.reader.beginArray();
   }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapred/BatchInputSplit.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/mapred/BatchInputSplit.java
deleted file mode 100644
index db69081..0000000
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapred/BatchInputSplit.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.clientImpl.mapred;
-
-import java.io.IOException;
-import java.util.Collection;
-
-import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.data.TableId;
-import org.apache.hadoop.mapred.InputSplit;
-
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
-/**
- * The Class BatchInputSplit. Encapsulates Accumulo ranges for use in Map Reduce jobs. Can contain
- * several Ranges per InputSplit.
- *
- * @deprecated since 2.0.0
- */
-@SuppressFBWarnings(value = "NM_SAME_SIMPLE_NAME_AS_SUPERCLASS",
-    justification = "Intended to share code between mapred and mapreduce")
-@Deprecated(since = "2.0.0")
-public class BatchInputSplit extends org.apache.accumulo.core.clientImpl.mapreduce.BatchInputSplit
-    implements InputSplit {
-
-  public BatchInputSplit() {}
-
-  public BatchInputSplit(BatchInputSplit split) throws IOException {
-    super(split);
-  }
-
-  public BatchInputSplit(String table, TableId tableId, Collection<Range> ranges,
-      String[] location) {
-    super(table, tableId, ranges, location);
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/BatchInputSplit.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/BatchInputSplit.java
deleted file mode 100644
index cc01471..0000000
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/BatchInputSplit.java
+++ /dev/null
@@ -1,200 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.clientImpl.mapreduce;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.PartialKey;
-import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.data.TableId;
-
-/**
- * The Class BatchInputSplit. Encapsulates a set of Accumulo ranges on a single tablet for use in
- * Map Reduce jobs. Can contain several Ranges per split.
- *
- * @deprecated since 2.0.0
- */
-@Deprecated(since = "2.0.0")
-public class BatchInputSplit extends org.apache.accumulo.core.client.mapreduce.RangeInputSplit {
-  private Collection<Range> ranges;
-  private float[] rangeProgress = null;
-
-  public BatchInputSplit() {
-    ranges = Collections.emptyList();
-  }
-
-  public BatchInputSplit(BatchInputSplit split) throws IOException {
-    super(split);
-    this.setRanges(split.getRanges());
-  }
-
-  public BatchInputSplit(String table, TableId tableId, Collection<Range> ranges,
-      String[] locations) {
-    super(table, tableId.canonical(), new Range(), locations);
-    this.ranges = ranges;
-  }
-
-  /**
-   * Save progress on each call to this function, implied by value of currentKey, and return average
-   * ranges in the split
-   */
-  @Override
-  public float getProgress(Key currentKey) {
-    if (rangeProgress == null) {
-      rangeProgress = new float[ranges.size()];
-    }
-
-    float total = 0; // progress per range could be on different scales, this number is "fuzzy"
-
-    if (currentKey == null) {
-      for (float progress : rangeProgress) {
-        total += progress;
-      }
-    } else {
-      int i = 0;
-      for (Range range : ranges) {
-        if (range.contains(currentKey)) {
-          // find the current range and report as if that is the single range
-          if (range.getStartKey() != null && range.getEndKey() != null) {
-            if (range.getStartKey().compareTo(range.getEndKey(), PartialKey.ROW) != 0) {
-              // just look at the row progress
-              rangeProgress[i] = SplitUtils.getProgress(range.getStartKey().getRowData(),
-                  range.getEndKey().getRowData(), currentKey.getRowData());
-            } else if (range.getStartKey().compareTo(range.getEndKey(), PartialKey.ROW_COLFAM)
-                != 0) {
-              // just look at the column family progress
-              rangeProgress[i] = SplitUtils.getProgress(range.getStartKey().getColumnFamilyData(),
-                  range.getEndKey().getColumnFamilyData(), currentKey.getColumnFamilyData());
-            } else if (range.getStartKey().compareTo(range.getEndKey(),
-                PartialKey.ROW_COLFAM_COLQUAL) != 0) {
-              // just look at the column qualifier progress
-              rangeProgress[i] = SplitUtils.getProgress(
-                  range.getStartKey().getColumnQualifierData(),
-                  range.getEndKey().getColumnQualifierData(), currentKey.getColumnQualifierData());
-            }
-          }
-          total += rangeProgress[i];
-        }
-        i++;
-      }
-    }
-
-    return total / ranges.size();
-  }
-
-  /**
-   * This implementation of length is only an estimate, it does not provide exact values. Do not
-   * have your code rely on this return value.
-   */
-  @Override
-  public long getLength() {
-    long sum = 0;
-    for (Range range : ranges) {
-      sum += SplitUtils.getRangeLength(range);
-    }
-    return sum;
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    super.readFields(in);
-
-    int numRanges = in.readInt();
-    ranges = new ArrayList<>(numRanges);
-    for (int i = 0; i < numRanges; ++i) {
-      Range r = new Range();
-      r.readFields(in);
-      ranges.add(r);
-    }
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    super.write(out);
-
-    out.writeInt(ranges.size());
-    for (Range r : ranges) {
-      r.write(out);
-    }
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder sb = new StringBuilder(256);
-    sb.append("BatchInputSplit:");
-    sb.append(" Ranges: ").append(Arrays.asList(ranges));
-    sb.append(super.toString());
-    return sb.toString();
-  }
-
-  public void setRanges(Collection<Range> ranges) {
-    this.ranges = ranges;
-  }
-
-  public Collection<Range> getRanges() {
-    return ranges;
-  }
-
-  @Override
-  public Range getRange() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void setRange(Range range) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public Boolean isIsolatedScan() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void setIsolatedScan(Boolean isolatedScan) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public Boolean isOffline() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void setOffline(Boolean offline) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public Boolean usesLocalIterators() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void setUsesLocalIterators(Boolean localIterators) {
-    throw new UnsupportedOperationException();
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/DelegationTokenStub.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/DelegationTokenStub.java
deleted file mode 100644
index 7871624..0000000
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/DelegationTokenStub.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.clientImpl.mapreduce;
-
-import static java.util.Objects.requireNonNull;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.util.Set;
-
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-
-/**
- * An internal stub class for passing DelegationToken information out of the Configuration back up
- * to the appropriate implementation for mapreduce or mapred.
- *
- * @deprecated since 2.0.0
- */
-@Deprecated(since = "2.0.0")
-public class DelegationTokenStub implements AuthenticationToken {
-
-  private String serviceName;
-
-  public DelegationTokenStub(String serviceName) {
-    requireNonNull(serviceName);
-    this.serviceName = serviceName;
-  }
-
-  public String getServiceName() {
-    return serviceName;
-  }
-
-  @Override
-  public void write(DataOutput out) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void readFields(DataInput in) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void destroy() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public boolean isDestroyed() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void init(Properties properties) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public Set<TokenProperty> getProperties() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public AuthenticationToken clone() {
-    throw new UnsupportedOperationException();
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/SplitUtils.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/SplitUtils.java
deleted file mode 100644
index 9634659..0000000
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/SplitUtils.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.clientImpl.mapreduce;
-
-import java.math.BigInteger;
-
-import org.apache.accumulo.core.data.ByteSequence;
-import org.apache.accumulo.core.data.Range;
-import org.apache.hadoop.io.Text;
-import org.apache.log4j.Level;
-
-/**
- * @deprecated since 2.0.0
- */
-@Deprecated(since = "2.0.0")
-public class SplitUtils {
-
-  /**
-   * Central place to set common split configuration not handled by split constructors. The
-   * intention is to make it harder to miss optional setters in future refactor.
-   */
-  public static void updateSplit(org.apache.accumulo.core.client.mapreduce.RangeInputSplit split,
-      org.apache.accumulo.core.client.mapreduce.InputTableConfig tableConfig, Level logLevel) {
-    split.setFetchedColumns(tableConfig.getFetchedColumns());
-    split.setIterators(tableConfig.getIterators());
-    split.setLogLevel(logLevel);
-    split.setSamplerConfiguration(tableConfig.getSamplerConfiguration());
-  }
-
-  public static float getProgress(ByteSequence start, ByteSequence end, ByteSequence position) {
-    int maxDepth = Math.min(Math.max(end.length(), start.length()), position.length());
-    BigInteger startBI = new BigInteger(SplitUtils.extractBytes(start, maxDepth));
-    BigInteger endBI = new BigInteger(SplitUtils.extractBytes(end, maxDepth));
-    BigInteger positionBI = new BigInteger(SplitUtils.extractBytes(position, maxDepth));
-    return (float) (positionBI.subtract(startBI).doubleValue()
-        / endBI.subtract(startBI).doubleValue());
-  }
-
-  public static long getRangeLength(Range range) {
-    Text startRow = range.isInfiniteStartKey() ? new Text(new byte[] {Byte.MIN_VALUE})
-        : range.getStartKey().getRow();
-    Text stopRow = range.isInfiniteStopKey() ? new Text(new byte[] {Byte.MAX_VALUE})
-        : range.getEndKey().getRow();
-    int maxCommon = Math.min(7, Math.min(startRow.getLength(), stopRow.getLength()));
-    long diff = 0;
-
-    byte[] start = startRow.getBytes();
-    byte[] stop = stopRow.getBytes();
-    for (int i = 0; i < maxCommon; ++i) {
-      diff |= 0xff & (start[i] ^ stop[i]);
-      diff <<= Byte.SIZE;
-    }
-
-    if (startRow.getLength() != stopRow.getLength()) {
-      diff |= 0xff;
-    }
-
-    return diff + 1;
-  }
-
-  static byte[] extractBytes(ByteSequence seq, int numBytes) {
-    byte[] bytes = new byte[numBytes + 1];
-    bytes[0] = 0;
-    for (int i = 0; i < numBytes; i++) {
-      if (i >= seq.length()) {
-        bytes[i + 1] = 0;
-      } else {
-        bytes[i + 1] = seq.byteAt(i);
-      }
-    }
-    return bytes;
-  }
-
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/ConfiguratorBase.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/ConfiguratorBase.java
deleted file mode 100644
index 6d4b3c6..0000000
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/ConfiguratorBase.java
+++ /dev/null
@@ -1,520 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.clientImpl.mapreduce.lib;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static java.nio.charset.StandardCharsets.UTF_8;
-import static java.util.Objects.requireNonNull;
-
-import java.io.ByteArrayInputStream;
-import java.io.DataInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.Base64;
-import java.util.Scanner;
-
-import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken.AuthenticationTokenSerializer;
-import org.apache.accumulo.core.clientImpl.AuthenticationTokenIdentifier;
-import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.clientImpl.Credentials;
-import org.apache.accumulo.core.clientImpl.DelegationTokenImpl;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.security.token.TokenIdentifier;
-import org.apache.hadoop.util.StringUtils;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
-
-@SuppressWarnings("deprecation")
-public class ConfiguratorBase {
-
-  protected static final Logger log = Logger.getLogger(ConfiguratorBase.class);
-
-  /**
-   * Specifies that connection info was configured
-   *
-   * @since 1.6.0
-   */
-  public enum ConnectorInfo {
-    IS_CONFIGURED, PRINCIPAL, TOKEN
-  }
-
-  public enum TokenSource {
-    FILE, INLINE, JOB;
-
-    private String prefix;
-
-    private TokenSource() {
-      prefix = name().toLowerCase() + ":";
-    }
-
-    public String prefix() {
-      return prefix;
-    }
-  }
-
-  /**
-   * Configuration keys for available Instance types.
-   *
-   * @since 1.6.0
-   */
-  public enum InstanceOpts {
-    TYPE, NAME, ZOO_KEEPERS, CLIENT_CONFIG
-  }
-
-  /**
-   * Configuration keys for general configuration options.
-   *
-   * @since 1.6.0
-   */
-  public enum GeneralOpts {
-    LOG_LEVEL, VISIBILITY_CACHE_SIZE
-  }
-
-  /**
-   * Provides a configuration key for a given feature enum, prefixed by the implementingClass
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param e the enum used to provide the unique part of the configuration key
-   * @return the configuration key
-   * @since 1.6.0
-   */
-  protected static String enumToConfKey(Class<?> implementingClass, Enum<?> e) {
-    return implementingClass.getSimpleName() + "." + e.getDeclaringClass().getSimpleName() + "."
-        + StringUtils.camelize(e.name().toLowerCase());
-  }
-
-  /**
-   * Provides a configuration key for a given feature enum.
-   *
-   * @param e the enum used to provide the unique part of the configuration key
-   * @return the configuration key
-   */
-  protected static String enumToConfKey(Enum<?> e) {
-    return e.getDeclaringClass().getSimpleName() + "."
-        + StringUtils.camelize(e.name().toLowerCase());
-  }
-
-  /**
-   * Sets the connector information needed to communicate with Accumulo in this job.
-   *
-   * <p>
-   * <b>WARNING:</b> The serialized token is stored in the configuration and shared with all
-   * MapReduce tasks. It is BASE64 encoded to provide a charset safe conversion to a string, and is
-   * not intended to be secure.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @param principal a valid Accumulo user name
-   * @param token the user's password
-   * @since 1.6.0
-   */
-  public static void setConnectorInfo(Class<?> implementingClass, Configuration conf,
-      String principal, AuthenticationToken token) {
-    if (isConnectorInfoSet(implementingClass, conf)) {
-      throw new IllegalStateException("Connector info for " + implementingClass.getSimpleName()
-          + " can only be set once per job");
-    }
-    checkArgument(principal != null, "principal is null");
-    checkArgument(token != null, "token is null");
-    conf.setBoolean(enumToConfKey(implementingClass, ConnectorInfo.IS_CONFIGURED), true);
-    conf.set(enumToConfKey(implementingClass, ConnectorInfo.PRINCIPAL), principal);
-    if (token instanceof DelegationTokenImpl) {
-      // Avoid serializing the DelegationToken secret in the configuration -- the Job will do that
-      // work for us securely
-      DelegationTokenImpl delToken = (DelegationTokenImpl) token;
-      conf.set(enumToConfKey(implementingClass, ConnectorInfo.TOKEN), TokenSource.JOB.prefix()
-          + token.getClass().getName() + ":" + delToken.getServiceName().toString());
-    } else {
-      conf.set(enumToConfKey(implementingClass, ConnectorInfo.TOKEN),
-          TokenSource.INLINE.prefix() + token.getClass().getName() + ":"
-              + Base64.getEncoder().encodeToString(AuthenticationTokenSerializer.serialize(token)));
-    }
-  }
-
-  private static String cachedTokenFileName(Class<?> implementingClass) {
-    return implementingClass.getSimpleName() + ".tokenfile";
-  }
-
-  /**
-   * Sets the connector information needed to communicate with Accumulo in this job.
-   *
-   * <p>
-   * Pulls a token file into the Distributed Cache that contains the authentication token in an
-   * attempt to be more secure than storing the password in the Configuration. Token file created
-   * with "bin/accumulo create-token".
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @param principal a valid Accumulo user name
-   * @param tokenFile the path to the token file in DFS
-   * @since 1.6.0
-   */
-  public static void setConnectorInfo(Class<?> implementingClass, Configuration conf,
-      String principal, String tokenFile) {
-    if (isConnectorInfoSet(implementingClass, conf)) {
-      throw new IllegalStateException("Connector info for " + implementingClass.getSimpleName()
-          + " can only be set once per job");
-    }
-
-    checkArgument(principal != null, "principal is null");
-    checkArgument(tokenFile != null, "tokenFile is null");
-
-    DistributedCacheHelper.addCacheFile(tokenFile, cachedTokenFileName(implementingClass), conf);
-
-    conf.setBoolean(enumToConfKey(implementingClass, ConnectorInfo.IS_CONFIGURED), true);
-    conf.set(enumToConfKey(implementingClass, ConnectorInfo.PRINCIPAL), principal);
-    conf.set(enumToConfKey(implementingClass, ConnectorInfo.TOKEN),
-        TokenSource.FILE.prefix() + tokenFile);
-  }
-
-  /**
-   * Determines if the connector info has already been set for this instance.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @return true if the connector info has already been set, false otherwise
-   * @since 1.6.0
-   * @see #setConnectorInfo(Class, Configuration, String, AuthenticationToken)
-   */
-  public static Boolean isConnectorInfoSet(Class<?> implementingClass, Configuration conf) {
-    return conf.getBoolean(enumToConfKey(implementingClass, ConnectorInfo.IS_CONFIGURED), false);
-  }
-
-  /**
-   * Gets the user name from the configuration.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @return the principal
-   * @since 1.6.0
-   * @see #setConnectorInfo(Class, Configuration, String, AuthenticationToken)
-   */
-  public static String getPrincipal(Class<?> implementingClass, Configuration conf) {
-    return conf.get(enumToConfKey(implementingClass, ConnectorInfo.PRINCIPAL));
-  }
-
-  /**
-   * Gets the authenticated token from either the specified token file or directly from the
-   * configuration, whichever was used when the job was configured.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @return the principal's authentication token
-   * @since 1.6.0
-   * @see #setConnectorInfo(Class, Configuration, String, AuthenticationToken)
-   * @see #setConnectorInfo(Class, Configuration, String, String)
-   */
-  public static AuthenticationToken getAuthenticationToken(Class<?> implementingClass,
-      Configuration conf) {
-    String token = conf.get(enumToConfKey(implementingClass, ConnectorInfo.TOKEN));
-    if (token == null || token.isEmpty()) {
-      return null;
-    }
-    if (token.startsWith(TokenSource.INLINE.prefix())) {
-      String[] args = token.substring(TokenSource.INLINE.prefix().length()).split(":", 2);
-      if (args.length == 2) {
-        return AuthenticationTokenSerializer.deserialize(args[0],
-            Base64.getDecoder().decode(args[1]));
-      }
-    } else if (token.startsWith(TokenSource.FILE.prefix())) {
-      String tokenFileName = token.substring(TokenSource.FILE.prefix().length());
-      return getTokenFromFile(implementingClass, conf, getPrincipal(implementingClass, conf),
-          tokenFileName);
-    } else if (token.startsWith(TokenSource.JOB.prefix())) {
-      String[] args = token.substring(TokenSource.JOB.prefix().length()).split(":", 2);
-      if (args.length == 2) {
-        String className = args[0], serviceName = args[1];
-        if (DelegationTokenImpl.class.getName().equals(className)) {
-          return new org.apache.accumulo.core.clientImpl.mapreduce.DelegationTokenStub(serviceName);
-        }
-      }
-    }
-
-    throw new IllegalStateException("Token was not properly serialized into the configuration");
-  }
-
-  /**
-   * Reads from the token file in distributed cache. Currently, the token file stores data separated
-   * by colons e.g. principal:token_class:token
-   *
-   * @param conf the Hadoop context for the configured job
-   * @return path to the token file as a String
-   * @since 1.6.0
-   * @see #setConnectorInfo(Class, Configuration, String, AuthenticationToken)
-   */
-  public static AuthenticationToken getTokenFromFile(Class<?> implementingClass, Configuration conf,
-      String principal, String tokenFile) {
-
-    try (InputStream inputStream = DistributedCacheHelper.openCachedFile(tokenFile,
-        cachedTokenFileName(implementingClass), conf)) {
-
-      try (Scanner fileScanner = new Scanner(inputStream, UTF_8)) {
-        while (fileScanner.hasNextLine()) {
-          Credentials creds = Credentials.deserialize(fileScanner.nextLine());
-          if (principal.equals(creds.getPrincipal())) {
-            return creds.getToken();
-          }
-        }
-        throw new IllegalArgumentException("No token found for " + principal);
-      }
-
-    } catch (IOException e) {
-      throw new IllegalStateException("Error closing token file stream", e);
-    }
-
-  }
-
-  /**
-   * Configures a ZooKeeperInstance for this job.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @param clientConfig client configuration for specifying connection timeouts, SSL connection
-   *        options, etc.
-   * @since 1.6.0
-   */
-  public static void setZooKeeperInstance(Class<?> implementingClass, Configuration conf,
-      org.apache.accumulo.core.client.ClientConfiguration clientConfig) {
-    String key = enumToConfKey(implementingClass, InstanceOpts.TYPE);
-    if (!conf.get(key, "").isEmpty()) {
-      throw new IllegalStateException(
-          "Instance info can only be set once per job; it has already been configured with "
-              + conf.get(key));
-    }
-    conf.set(key, "ZooKeeperInstance");
-    if (clientConfig != null) {
-      conf.set(enumToConfKey(implementingClass, InstanceOpts.CLIENT_CONFIG),
-          clientConfig.serialize());
-    }
-  }
-
-  /**
-   * Initializes an Accumulo Instance based on the configuration.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @return an Accumulo instance
-   * @since 1.6.0
-   */
-  public static org.apache.accumulo.core.client.Instance getInstance(Class<?> implementingClass,
-      Configuration conf) {
-    String instanceType = conf.get(enumToConfKey(implementingClass, InstanceOpts.TYPE), "");
-    if ("ZooKeeperInstance".equals(instanceType)) {
-      return new org.apache.accumulo.core.client.ZooKeeperInstance(
-          getClientConfiguration(implementingClass, conf));
-    } else if (instanceType.isEmpty()) {
-      throw new IllegalStateException(
-          "Instance has not been configured for " + implementingClass.getSimpleName());
-    } else {
-      throw new IllegalStateException("Unrecognized instance type " + instanceType);
-    }
-  }
-
-  /**
-   * Obtain a ClientConfiguration based on the configuration.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   *
-   * @return A ClientConfiguration
-   * @since 1.7.0
-   */
-  public static org.apache.accumulo.core.client.ClientConfiguration
-      getClientConfiguration(Class<?> implementingClass, Configuration conf) {
-    String clientConfigString =
-        conf.get(enumToConfKey(implementingClass, InstanceOpts.CLIENT_CONFIG));
-    if (null != clientConfigString) {
-      return org.apache.accumulo.core.client.ClientConfiguration.deserialize(clientConfigString);
-    }
-
-    String instanceName = conf.get(enumToConfKey(implementingClass, InstanceOpts.NAME));
-    String zookeepers = conf.get(enumToConfKey(implementingClass, InstanceOpts.ZOO_KEEPERS));
-    org.apache.accumulo.core.client.ClientConfiguration clientConf =
-        org.apache.accumulo.core.client.ClientConfiguration.loadDefault();
-    if (null != instanceName) {
-      clientConf.withInstance(instanceName);
-    }
-    if (null != zookeepers) {
-      clientConf.withZkHosts(zookeepers);
-    }
-    return clientConf;
-  }
-
-  /**
-   * Sets the log level for this job.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @param level the logging level
-   * @since 1.6.0
-   */
-  public static void setLogLevel(Class<?> implementingClass, Configuration conf, Level level) {
-    checkArgument(level != null, "level is null");
-    Logger.getLogger(implementingClass).setLevel(level);
-    conf.setInt(enumToConfKey(implementingClass, GeneralOpts.LOG_LEVEL), level.toInt());
-  }
-
-  /**
-   * Gets the log level from this configuration.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @return the log level
-   * @since 1.6.0
-   * @see #setLogLevel(Class, Configuration, Level)
-   */
-  public static Level getLogLevel(Class<?> implementingClass, Configuration conf) {
-    return Level.toLevel(
-        conf.getInt(enumToConfKey(implementingClass, GeneralOpts.LOG_LEVEL), Level.INFO.toInt()));
-  }
-
-  /**
-   * Sets the valid visibility count for this job.
-   *
-   * @param conf the Hadoop configuration object to configure
-   * @param visibilityCacheSize the LRU cache size
-   */
-  public static void setVisibilityCacheSize(Configuration conf, int visibilityCacheSize) {
-    conf.setInt(enumToConfKey(GeneralOpts.VISIBILITY_CACHE_SIZE), visibilityCacheSize);
-  }
-
-  /**
-   * Gets the valid visibility count for this job.
-   *
-   * @param conf the Hadoop configuration object to configure
-   * @return the valid visibility count
-   */
-  public static int getVisibilityCacheSize(Configuration conf) {
-    return conf.getInt(enumToConfKey(GeneralOpts.VISIBILITY_CACHE_SIZE),
-        Constants.DEFAULT_VISIBILITY_CACHE_SIZE);
-  }
-
-  /**
-   * Unwraps the provided {@link AuthenticationToken} if it is an instance of DelegationTokenStub,
-   * reconstituting it from the provided {@link JobConf}.
-   *
-   * @param job The job
-   * @param token The authentication token
-   */
-  public static AuthenticationToken unwrapAuthenticationToken(JobConf job,
-      AuthenticationToken token) {
-    requireNonNull(job);
-    requireNonNull(token);
-    if (token instanceof org.apache.accumulo.core.clientImpl.mapreduce.DelegationTokenStub) {
-      org.apache.accumulo.core.clientImpl.mapreduce.DelegationTokenStub delTokenStub =
-          (org.apache.accumulo.core.clientImpl.mapreduce.DelegationTokenStub) token;
-      Token<? extends TokenIdentifier> hadoopToken =
-          job.getCredentials().getToken(new Text(delTokenStub.getServiceName()));
-      AuthenticationTokenIdentifier identifier = new AuthenticationTokenIdentifier();
-      try {
-        identifier
-            .readFields(new DataInputStream(new ByteArrayInputStream(hadoopToken.getIdentifier())));
-        return new DelegationTokenImpl(hadoopToken.getPassword(), identifier);
-      } catch (IOException e) {
-        throw new RuntimeException("Could not construct DelegationToken from JobConf Credentials",
-            e);
-      }
-    }
-    return token;
-  }
-
-  /**
-   * Unwraps the provided {@link AuthenticationToken} if it is an instance of DelegationTokenStub,
-   * reconstituting it from the provided {@link JobConf}.
-   *
-   * @param job The job
-   * @param token The authentication token
-   */
-  public static AuthenticationToken unwrapAuthenticationToken(JobContext job,
-      AuthenticationToken token) {
-    requireNonNull(job);
-    requireNonNull(token);
-    if (token instanceof org.apache.accumulo.core.clientImpl.mapreduce.DelegationTokenStub) {
-      org.apache.accumulo.core.clientImpl.mapreduce.DelegationTokenStub delTokenStub =
-          (org.apache.accumulo.core.clientImpl.mapreduce.DelegationTokenStub) token;
-      Token<? extends TokenIdentifier> hadoopToken =
-          job.getCredentials().getToken(new Text(delTokenStub.getServiceName()));
-      AuthenticationTokenIdentifier identifier = new AuthenticationTokenIdentifier();
-      try {
-        identifier
-            .readFields(new DataInputStream(new ByteArrayInputStream(hadoopToken.getIdentifier())));
-        return new DelegationTokenImpl(hadoopToken.getPassword(), identifier);
-      } catch (IOException e) {
-        throw new RuntimeException("Could not construct DelegationToken from JobConf Credentials",
-            e);
-      }
-    }
-    return token;
-  }
-
-  public static ClientContext client(Class<?> CLASS, Configuration conf)
-      throws AccumuloException, AccumuloSecurityException {
-    return ((org.apache.accumulo.core.clientImpl.ConnectorImpl) getInstance(CLASS, conf)
-        .getConnector(getPrincipal(CLASS, conf), getAuthenticationToken(CLASS, conf)))
-        .getAccumuloClient();
-  }
-
-  public static ClientContext client(Class<?> CLASS,
-      org.apache.accumulo.core.client.mapreduce.RangeInputSplit split, Configuration conf)
-      throws IOException {
-    try {
-      org.apache.accumulo.core.client.Instance instance =
-          split.getInstance(getClientConfiguration(CLASS, conf));
-      if (instance == null) {
-        instance = getInstance(CLASS, conf);
-      }
-
-      String principal = split.getPrincipal();
-      if (principal == null) {
-        principal = getPrincipal(CLASS, conf);
-      }
-
-      AuthenticationToken token = split.getToken();
-      if (token == null) {
-        token = getAuthenticationToken(CLASS, conf);
-      }
-
-      return ((org.apache.accumulo.core.clientImpl.ConnectorImpl) instance.getConnector(principal,
-          token)).getAccumuloClient();
-    } catch (AccumuloException | AccumuloSecurityException e) {
-      throw new IOException(e);
-    }
-
-  }
-
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/FileOutputConfigurator.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/FileOutputConfigurator.java
deleted file mode 100644
index c9a622f..0000000
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/FileOutputConfigurator.java
+++ /dev/null
@@ -1,242 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.clientImpl.mapreduce.lib;
-
-import java.util.Arrays;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-
-import org.apache.accumulo.core.client.sample.SamplerConfiguration;
-import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.ConfigurationCopy;
-import org.apache.accumulo.core.conf.DefaultConfiguration;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
-import org.apache.hadoop.conf.Configuration;
-
-/**
- * @since 1.6.0
- */
-public class FileOutputConfigurator extends ConfiguratorBase {
-
-  /**
-   * Configuration keys for {@link AccumuloConfiguration}.
-   *
-   * @since 1.6.0
-   */
-  public static enum Opts {
-    ACCUMULO_PROPERTIES
-  }
-
-  /**
-   * The supported Accumulo properties we set in this OutputFormat, that change the behavior of the
-   * RecordWriter.<br>
-   * These properties correspond to the supported public static setter methods available to this
-   * class.
-   *
-   * @param property the Accumulo property to check
-   * @since 1.6.0
-   */
-  protected static Boolean isSupportedAccumuloProperty(Property property) {
-    switch (property) {
-      case TABLE_FILE_COMPRESSION_TYPE:
-      case TABLE_FILE_COMPRESSED_BLOCK_SIZE:
-      case TABLE_FILE_BLOCK_SIZE:
-      case TABLE_FILE_COMPRESSED_BLOCK_SIZE_INDEX:
-      case TABLE_FILE_REPLICATION:
-        return true;
-      default:
-        return false;
-    }
-  }
-
-  /**
-   * Helper for transforming Accumulo configuration properties into something that can be stored
-   * safely inside the Hadoop Job configuration.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @param property the supported Accumulo property
-   * @param value the value of the property to set
-   * @since 1.6.0
-   */
-  private static <T> void setAccumuloProperty(Class<?> implementingClass, Configuration conf,
-      Property property, T value) {
-    if (isSupportedAccumuloProperty(property)) {
-      String val = String.valueOf(value);
-      if (property.getType().isValidFormat(val)) {
-        conf.set(
-            enumToConfKey(implementingClass, Opts.ACCUMULO_PROPERTIES) + "." + property.getKey(),
-            val);
-      } else {
-        throw new IllegalArgumentException(
-            "Value is not appropriate for property type '" + property.getType() + "'");
-      }
-    } else {
-      throw new IllegalArgumentException("Unsupported configuration property " + property.getKey());
-    }
-  }
-
-  /**
-   * This helper method provides an AccumuloConfiguration object constructed from the Accumulo
-   * defaults, and overridden with Accumulo properties that have been stored in the Job's
-   * configuration.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @since 1.6.0
-   */
-  public static AccumuloConfiguration getAccumuloConfiguration(Class<?> implementingClass,
-      Configuration conf) {
-    String prefix = enumToConfKey(implementingClass, Opts.ACCUMULO_PROPERTIES) + ".";
-    ConfigurationCopy acuConf = new ConfigurationCopy(DefaultConfiguration.getInstance());
-    for (Entry<String,String> entry : conf) {
-      if (entry.getKey().startsWith(prefix)) {
-        String propString = entry.getKey().substring(prefix.length());
-        Property prop = Property.getPropertyByKey(propString);
-        if (prop != null) {
-          acuConf.set(prop, entry.getValue());
-        } else if (Property.isValidTablePropertyKey(propString)) {
-          acuConf.set(propString, entry.getValue());
-        } else {
-          throw new IllegalArgumentException("Unknown accumulo file property " + propString);
-        }
-      }
-    }
-    return acuConf;
-  }
-
-  /**
-   * Sets the compression type to use for data blocks. Specifying a compression may require
-   * additional libraries to be available to your Job.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @param compressionType one of "none", "gz", "bzip2", "lzo", "lz4", "snappy", or "zstd"
-   * @since 1.6.0
-   */
-  public static void setCompressionType(Class<?> implementingClass, Configuration conf,
-      String compressionType) {
-    if (compressionType == null || !Arrays
-        .asList("none", "gz", "bzip2", "lzo", "lz4", "snappy", "zstd").contains(compressionType)) {
-      throw new IllegalArgumentException(
-          "Compression type must be one of: none, gz, bzip2, lzo, lz4, snappy, zstd");
-    }
-    setAccumuloProperty(implementingClass, conf, Property.TABLE_FILE_COMPRESSION_TYPE,
-        compressionType);
-  }
-
-  /**
-   * Sets the size for data blocks within each file.<br>
-   * Data blocks are a span of key/value pairs stored in the file that are compressed and indexed as
-   * a group.
-   *
-   * <p>
-   * Making this value smaller may increase seek performance, but at the cost of increasing the size
-   * of the indexes (which can also affect seek performance).
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @param dataBlockSize the block size, in bytes
-   * @since 1.6.0
-   */
-  public static void setDataBlockSize(Class<?> implementingClass, Configuration conf,
-      long dataBlockSize) {
-    setAccumuloProperty(implementingClass, conf, Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE,
-        dataBlockSize);
-  }
-
-  /**
-   * Sets the size for file blocks in the file system; file blocks are managed, and replicated, by
-   * the underlying file system.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @param fileBlockSize the block size, in bytes
-   * @since 1.6.0
-   */
-  public static void setFileBlockSize(Class<?> implementingClass, Configuration conf,
-      long fileBlockSize) {
-    setAccumuloProperty(implementingClass, conf, Property.TABLE_FILE_BLOCK_SIZE, fileBlockSize);
-  }
-
-  /**
-   * Sets the size for index blocks within each file; smaller blocks means a deeper index hierarchy
-   * within the file, while larger blocks mean a more shallow index hierarchy within the file. This
-   * can affect the performance of queries.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @param indexBlockSize the block size, in bytes
-   * @since 1.6.0
-   */
-  public static void setIndexBlockSize(Class<?> implementingClass, Configuration conf,
-      long indexBlockSize) {
-    setAccumuloProperty(implementingClass, conf, Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE_INDEX,
-        indexBlockSize);
-  }
-
-  /**
-   * Sets the file system replication factor for the resulting file, overriding the file system
-   * default.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @param replication the number of replicas for produced files
-   * @since 1.6.0
-   */
-  public static void setReplication(Class<?> implementingClass, Configuration conf,
-      int replication) {
-    setAccumuloProperty(implementingClass, conf, Property.TABLE_FILE_REPLICATION, replication);
-  }
-
-  /**
-   * @since 1.8.0
-   */
-  public static void setSampler(Class<?> implementingClass, Configuration conf,
-      SamplerConfiguration samplerConfig) {
-    Map<String,String> props = new SamplerConfigurationImpl(samplerConfig).toTablePropertiesMap();
-
-    Set<Entry<String,String>> es = props.entrySet();
-    for (Entry<String,String> entry : es) {
-      conf.set(enumToConfKey(implementingClass, Opts.ACCUMULO_PROPERTIES) + "." + entry.getKey(),
-          entry.getValue());
-    }
-  }
-
-  public static void setSummarizers(Class<?> implementingClass, Configuration conf,
-      SummarizerConfiguration[] sumarizerConfigs) {
-    Map<String,String> props = SummarizerConfiguration.toTableProperties(sumarizerConfigs);
-
-    for (Entry<String,String> entry : props.entrySet()) {
-      conf.set(enumToConfKey(implementingClass, Opts.ACCUMULO_PROPERTIES) + "." + entry.getKey(),
-          entry.getValue());
-    }
-  }
-
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/InputConfigurator.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/InputConfigurator.java
deleted file mode 100644
index b2a6e43..0000000
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/InputConfigurator.java
+++ /dev/null
@@ -1,924 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.clientImpl.mapreduce.lib;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static java.nio.charset.StandardCharsets.UTF_8;
-import static java.util.Objects.requireNonNull;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Base64;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.StringTokenizer;
-
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.BatchScanner;
-import org.apache.accumulo.core.client.ClientSideIteratorScanner;
-import org.apache.accumulo.core.client.IsolatedScanner;
-import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.RowIterator;
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.sample.SamplerConfiguration;
-import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.PartialKey;
-import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.dataImpl.KeyExtent;
-import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
-import org.apache.accumulo.core.manager.state.tables.TableState;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.CurrentLocationColumnFamily;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.FutureLocationColumnFamily;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LastLocationColumnFamily;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.TabletColumnFamily;
-import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.NamespacePermission;
-import org.apache.accumulo.core.security.TablePermission;
-import org.apache.accumulo.core.util.Pair;
-import org.apache.accumulo.core.util.TextUtil;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.MapWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.util.StringUtils;
-
-import com.google.common.collect.Maps;
-
-@SuppressWarnings("deprecation")
-public class InputConfigurator extends ConfiguratorBase {
-
-  /**
-   * Configuration keys for {@link Scanner}.
-   *
-   * @since 1.6.0
-   */
-  public enum ScanOpts {
-    TABLE_NAME,
-    AUTHORIZATIONS,
-    RANGES,
-    COLUMNS,
-    ITERATORS,
-    TABLE_CONFIGS,
-    SAMPLER_CONFIG,
-    CLASSLOADER_CONTEXT
-  }
-
-  /**
-   * Configuration keys for various features.
-   *
-   * @since 1.6.0
-   */
-  public enum Features {
-    AUTO_ADJUST_RANGES,
-    SCAN_ISOLATION,
-    USE_LOCAL_ITERATORS,
-    SCAN_OFFLINE,
-    BATCH_SCANNER,
-    BATCH_SCANNER_THREADS
-  }
-
-  /**
-   * Sets the name of the context classloader to use for scans
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @param context the name of the context classloader
-   * @since 1.8.0
-   */
-  public static void setClassLoaderContext(Class<?> implementingClass, Configuration conf,
-      String context) {
-    checkArgument(context != null, "context is null");
-    conf.set(enumToConfKey(implementingClass, ScanOpts.CLASSLOADER_CONTEXT), context);
-  }
-
-  /**
-   * Gets the name of the context classloader to use for scans
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @return the classloader context name
-   * @since 1.8.0
-   */
-  public static String getClassLoaderContext(Class<?> implementingClass, Configuration conf) {
-    return conf.get(enumToConfKey(implementingClass, ScanOpts.CLASSLOADER_CONTEXT), null);
-  }
-
-  /**
-   * Sets the name of the input table, over which this job will scan.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @param tableName the table to use when the tablename is null in the write call
-   * @since 1.6.0
-   */
-  public static void setInputTableName(Class<?> implementingClass, Configuration conf,
-      String tableName) {
-    checkArgument(tableName != null, "tableName is null");
-    conf.set(enumToConfKey(implementingClass, ScanOpts.TABLE_NAME), tableName);
-  }
-
-  /**
-   * Sets the name of the input table, over which this job will scan.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @since 1.6.0
-   */
-  public static String getInputTableName(Class<?> implementingClass, Configuration conf) {
-    return conf.get(enumToConfKey(implementingClass, ScanOpts.TABLE_NAME));
-  }
-
-  /**
-   * Sets the {@link Authorizations} used to scan. Must be a subset of the user's authorization.
-   * Defaults to the empty set.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @param auths the user's authorizations
-   * @since 1.6.0
-   */
-  public static void setScanAuthorizations(Class<?> implementingClass, Configuration conf,
-      Authorizations auths) {
-    if (auths != null && !auths.isEmpty()) {
-      conf.set(enumToConfKey(implementingClass, ScanOpts.AUTHORIZATIONS), auths.serialize());
-    }
-  }
-
-  /**
-   * Gets the authorizations to set for the scans from the configuration.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @return the Accumulo scan authorizations
-   * @since 1.6.0
-   * @see #setScanAuthorizations(Class, Configuration, Authorizations)
-   */
-  public static Authorizations getScanAuthorizations(Class<?> implementingClass,
-      Configuration conf) {
-    String authString = conf.get(enumToConfKey(implementingClass, ScanOpts.AUTHORIZATIONS));
-    return authString == null ? Authorizations.EMPTY
-        : new Authorizations(authString.getBytes(UTF_8));
-  }
-
-  /**
-   * Sets the input ranges to scan on all input tables for this job. If not set, the entire table
-   * will be scanned.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @param ranges the ranges that will be mapped over
-   * @throws IllegalArgumentException if the ranges cannot be encoded into base 64
-   * @since 1.6.0
-   */
-  public static void setRanges(Class<?> implementingClass, Configuration conf,
-      Collection<Range> ranges) {
-    checkArgument(ranges != null, "ranges is null");
-
-    ArrayList<String> rangeStrings = new ArrayList<>(ranges.size());
-    try {
-      for (Range r : ranges) {
-        ByteArrayOutputStream baos = new ByteArrayOutputStream();
-        r.write(new DataOutputStream(baos));
-        rangeStrings.add(Base64.getEncoder().encodeToString(baos.toByteArray()));
-      }
-      conf.setStrings(enumToConfKey(implementingClass, ScanOpts.RANGES),
-          rangeStrings.toArray(new String[0]));
-    } catch (IOException ex) {
-      throw new IllegalArgumentException("Unable to encode ranges to Base64", ex);
-    }
-  }
-
-  /**
-   * Gets the ranges to scan over from a job.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @return the ranges
-   * @throws IOException if the ranges have been encoded improperly
-   * @since 1.6.0
-   * @see #setRanges(Class, Configuration, Collection)
-   */
-  public static List<Range> getRanges(Class<?> implementingClass, Configuration conf)
-      throws IOException {
-
-    Collection<String> encodedRanges =
-        conf.getStringCollection(enumToConfKey(implementingClass, ScanOpts.RANGES));
-    List<Range> ranges = new ArrayList<>();
-    for (String rangeString : encodedRanges) {
-      ByteArrayInputStream bais = new ByteArrayInputStream(Base64.getDecoder().decode(rangeString));
-      Range range = new Range();
-      range.readFields(new DataInputStream(bais));
-      ranges.add(range);
-    }
-    return ranges;
-  }
-
-  /**
-   * Gets a list of the iterator settings (for iterators to apply to a scanner) from this
-   * configuration.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @return a list of iterators
-   * @since 1.6.0
-   * @see #addIterator(Class, Configuration, IteratorSetting)
-   */
-  public static List<IteratorSetting> getIterators(Class<?> implementingClass, Configuration conf) {
-    String iterators = conf.get(enumToConfKey(implementingClass, ScanOpts.ITERATORS));
-
-    // If no iterators are present, return an empty list
-    if (iterators == null || iterators.isEmpty()) {
-      return new ArrayList<>();
-    }
-
-    // Compose the set of iterators encoded in the job configuration
-    StringTokenizer tokens = new StringTokenizer(iterators, StringUtils.COMMA_STR);
-    List<IteratorSetting> list = new ArrayList<>();
-    try {
-      while (tokens.hasMoreTokens()) {
-        String itstring = tokens.nextToken();
-        ByteArrayInputStream bais = new ByteArrayInputStream(Base64.getDecoder().decode(itstring));
-        list.add(new IteratorSetting(new DataInputStream(bais)));
-        bais.close();
-      }
-    } catch (IOException e) {
-      throw new IllegalArgumentException("couldn't decode iterator settings");
-    }
-    return list;
-  }
-
-  /**
-   * Restricts the columns that will be mapped over for the single input table on this job.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @param columnFamilyColumnQualifierPairs a pair of {@link Text} objects corresponding to column
-   *        family and column qualifier. If the column qualifier is null, the entire column family
-   *        is selected. An empty set is the default and is equivalent to scanning the all columns.
-   * @throws IllegalArgumentException if the column family is null
-   * @since 1.6.0
-   */
-  public static void fetchColumns(Class<?> implementingClass, Configuration conf,
-      Collection<Pair<Text,Text>> columnFamilyColumnQualifierPairs) {
-    checkArgument(columnFamilyColumnQualifierPairs != null,
-        "columnFamilyColumnQualifierPairs is null");
-    String[] columnStrings = serializeColumns(columnFamilyColumnQualifierPairs);
-    conf.setStrings(enumToConfKey(implementingClass, ScanOpts.COLUMNS), columnStrings);
-  }
-
-  public static String[]
-      serializeColumns(Collection<Pair<Text,Text>> columnFamilyColumnQualifierPairs) {
-    checkArgument(columnFamilyColumnQualifierPairs != null,
-        "columnFamilyColumnQualifierPairs is null");
-    ArrayList<String> columnStrings = new ArrayList<>(columnFamilyColumnQualifierPairs.size());
-    for (Pair<Text,Text> column : columnFamilyColumnQualifierPairs) {
-
-      if (column.getFirst() == null) {
-        throw new IllegalArgumentException("Column family can not be null");
-      }
-
-      String col = Base64.getEncoder().encodeToString(TextUtil.getBytes(column.getFirst()));
-      if (column.getSecond() != null) {
-        col += ":" + Base64.getEncoder().encodeToString(TextUtil.getBytes(column.getSecond()));
-      }
-      columnStrings.add(col);
-    }
-
-    return columnStrings.toArray(new String[0]);
-  }
-
-  /**
-   * Gets the columns to be mapped over from this job.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @return a set of columns
-   * @since 1.6.0
-   * @see #fetchColumns(Class, Configuration, Collection)
-   */
-  public static Set<Pair<Text,Text>> getFetchedColumns(Class<?> implementingClass,
-      Configuration conf) {
-    checkArgument(conf != null, "conf is null");
-    String confValue = conf.get(enumToConfKey(implementingClass, ScanOpts.COLUMNS));
-    List<String> serialized = new ArrayList<>();
-    if (confValue != null) {
-      // Split and include any trailing empty strings to allow empty column families
-      Collections.addAll(serialized, confValue.split(",", -1));
-    }
-    return deserializeFetchedColumns(serialized);
-  }
-
-  public static Set<Pair<Text,Text>> deserializeFetchedColumns(Collection<String> serialized) {
-    Set<Pair<Text,Text>> columns = new HashSet<>();
-
-    if (serialized == null) {
-      return columns;
-    }
-
-    for (String col : serialized) {
-      int idx = col.indexOf(":");
-      Text cf = new Text(idx < 0 ? Base64.getDecoder().decode(col)
-          : Base64.getDecoder().decode(col.substring(0, idx)));
-      Text cq = idx < 0 ? null : new Text(Base64.getDecoder().decode(col.substring(idx + 1)));
-      columns.add(new Pair<>(cf, cq));
-    }
-    return columns;
-  }
-
-  /**
-   * Encode an iterator on the input for the single input table associated with this job.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @param cfg the configuration of the iterator
-   * @throws IllegalArgumentException if the iterator can't be serialized into the configuration
-   * @since 1.6.0
-   */
-  public static void addIterator(Class<?> implementingClass, Configuration conf,
-      IteratorSetting cfg) {
-    ByteArrayOutputStream baos = new ByteArrayOutputStream();
-    String newIter;
-    try {
-      cfg.write(new DataOutputStream(baos));
-      newIter = Base64.getEncoder().encodeToString(baos.toByteArray());
-      baos.close();
-    } catch (IOException e) {
-      throw new IllegalArgumentException("unable to serialize IteratorSetting");
-    }
-
-    String confKey = enumToConfKey(implementingClass, ScanOpts.ITERATORS);
-    String iterators = conf.get(confKey);
-    // No iterators specified yet, create a new string
-    if (iterators == null || iterators.isEmpty()) {
-      iterators = newIter;
-    } else {
-      // append the next iterator & reset
-      iterators = iterators.concat(StringUtils.COMMA_STR + newIter);
-    }
-    // Store the iterators w/ the job
-    conf.set(confKey, iterators);
-  }
-
-  /**
-   * Controls the automatic adjustment of ranges for this job. This feature merges overlapping
-   * ranges, then splits them to align with tablet boundaries. Disabling this feature will cause
-   * exactly one Map task to be created for each specified range. The default setting is enabled. *
-   *
-   * <p>
-   * By default, this feature is <b>enabled</b>.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @param enableFeature the feature is enabled if true, disabled otherwise
-   * @see #setRanges(Class, Configuration, Collection)
-   * @since 1.6.0
-   */
-  public static void setAutoAdjustRanges(Class<?> implementingClass, Configuration conf,
-      boolean enableFeature) {
-    conf.setBoolean(enumToConfKey(implementingClass, Features.AUTO_ADJUST_RANGES), enableFeature);
-  }
-
-  /**
-   * Determines whether a configuration has auto-adjust ranges enabled.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @return false if the feature is disabled, true otherwise
-   * @since 1.6.0
-   * @see #setAutoAdjustRanges(Class, Configuration, boolean)
-   */
-  public static Boolean getAutoAdjustRanges(Class<?> implementingClass, Configuration conf) {
-    return conf.getBoolean(enumToConfKey(implementingClass, Features.AUTO_ADJUST_RANGES), true);
-  }
-
-  /**
-   * Controls the use of the {@link IsolatedScanner} in this job.
-   *
-   * <p>
-   * By default, this feature is <b>disabled</b>.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @param enableFeature the feature is enabled if true, disabled otherwise
-   * @since 1.6.0
-   */
-  public static void setScanIsolation(Class<?> implementingClass, Configuration conf,
-      boolean enableFeature) {
-    conf.setBoolean(enumToConfKey(implementingClass, Features.SCAN_ISOLATION), enableFeature);
-  }
-
-  /**
-   * Determines whether a configuration has isolation enabled.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @return true if the feature is enabled, false otherwise
-   * @since 1.6.0
-   * @see #setScanIsolation(Class, Configuration, boolean)
-   */
-  public static Boolean isIsolated(Class<?> implementingClass, Configuration conf) {
-    return conf.getBoolean(enumToConfKey(implementingClass, Features.SCAN_ISOLATION), false);
-  }
-
-  /**
-   * Controls the use of the {@link ClientSideIteratorScanner} in this job. Enabling this feature
-   * will cause the iterator stack to be constructed within the Map task, rather than within the
-   * Accumulo TServer. To use this feature, all classes needed for those iterators must be available
-   * on the classpath for the task.
-   *
-   * <p>
-   * By default, this feature is <b>disabled</b>.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @param enableFeature the feature is enabled if true, disabled otherwise
-   * @since 1.6.0
-   */
-  public static void setLocalIterators(Class<?> implementingClass, Configuration conf,
-      boolean enableFeature) {
-    conf.setBoolean(enumToConfKey(implementingClass, Features.USE_LOCAL_ITERATORS), enableFeature);
-  }
-
-  /**
-   * Determines whether a configuration uses local iterators.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @return true if the feature is enabled, false otherwise
-   * @since 1.6.0
-   * @see #setLocalIterators(Class, Configuration, boolean)
-   */
-  public static Boolean usesLocalIterators(Class<?> implementingClass, Configuration conf) {
-    return conf.getBoolean(enumToConfKey(implementingClass, Features.USE_LOCAL_ITERATORS), false);
-  }
-
-  /**
-   * Enable reading offline tables. By default, this feature is disabled and only online tables are
-   * scanned. This will make the map reduce job directly read the table's files. If the table is not
-   * offline, then the job will fail. If the table comes online during the map reduce job, it is
-   * likely that the job will fail.
-   *
-   * <p>
-   * To use this option, the map reduce user will need access to read the Accumulo directory in
-   * HDFS.
-   *
-   * <p>
-   * Reading the offline table will create the scan time iterator stack in the map process. So any
-   * iterators that are configured for the table will need to be on the mapper's classpath.
-   *
-   * <p>
-   * One way to use this feature is to clone a table, take the clone offline, and use the clone as
-   * the input table for a map reduce job. If you plan to map reduce over the data many times, it
-   * may be better to the compact the table, clone it, take it offline, and use the clone for all
-   * map reduce jobs. The reason to do this is that compaction will reduce each tablet in the table
-   * to one file, and it is faster to read from one file.
-   *
-   * <p>
-   * There are two possible advantages to reading a tables file directly out of HDFS. First, you may
-   * see better read performance. Second, it will support speculative execution better. When reading
-   * an online table speculative execution can put more load on an already slow tablet server.
-   *
-   * <p>
-   * By default, this feature is <b>disabled</b>.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @param enableFeature the feature is enabled if true, disabled otherwise
-   * @since 1.6.0
-   */
-  public static void setOfflineTableScan(Class<?> implementingClass, Configuration conf,
-      boolean enableFeature) {
-    conf.setBoolean(enumToConfKey(implementingClass, Features.SCAN_OFFLINE), enableFeature);
-  }
-
-  /**
-   * Determines whether a configuration has the offline table scan feature enabled.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @return true if the feature is enabled, false otherwise
-   * @since 1.6.0
-   * @see #setOfflineTableScan(Class, Configuration, boolean)
-   */
-  public static Boolean isOfflineScan(Class<?> implementingClass, Configuration conf) {
-    return conf.getBoolean(enumToConfKey(implementingClass, Features.SCAN_OFFLINE), false);
-  }
-
-  /**
-   * Controls the use of the {@link BatchScanner} in this job. Using this feature will group ranges
-   * by their source tablet per InputSplit and use BatchScanner to read them.
-   *
-   * <p>
-   * By default, this feature is <b>disabled</b>.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @param enableFeature the feature is enabled if true, disabled otherwise
-   * @since 1.7.0
-   */
-  public static void setBatchScan(Class<?> implementingClass, Configuration conf,
-      boolean enableFeature) {
-    conf.setBoolean(enumToConfKey(implementingClass, Features.BATCH_SCANNER), enableFeature);
-  }
-
-  /**
-   * Determines whether a configuration has the BatchScanner feature enabled.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @return true if the feature is enabled, false otherwise
-   * @since 1.7.0
-   * @see #setBatchScan(Class, Configuration, boolean)
-   */
-  public static Boolean isBatchScan(Class<?> implementingClass, Configuration conf) {
-    return conf.getBoolean(enumToConfKey(implementingClass, Features.BATCH_SCANNER), false);
-  }
-
-  /**
-   * Sets configurations for multiple tables at a time.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @param configs an array of InputTableConfig objects to associate with the job
-   * @since 1.6.0
-   */
-  public static void setInputTableConfigs(Class<?> implementingClass, Configuration conf,
-      Map<String,org.apache.accumulo.core.client.mapreduce.InputTableConfig> configs) {
-    MapWritable mapWritable = new MapWritable();
-    for (Map.Entry<String,
-        org.apache.accumulo.core.client.mapreduce.InputTableConfig> tableConfig : configs
-            .entrySet()) {
-      mapWritable.put(new Text(tableConfig.getKey()), tableConfig.getValue());
-    }
-
-    ByteArrayOutputStream baos = new ByteArrayOutputStream();
-    try {
-      mapWritable.write(new DataOutputStream(baos));
-    } catch (IOException e) {
-      throw new IllegalStateException("Table configuration could not be serialized.");
-    }
-
-    String confKey = enumToConfKey(implementingClass, ScanOpts.TABLE_CONFIGS);
-    conf.set(confKey, Base64.getEncoder().encodeToString(baos.toByteArray()));
-  }
-
-  /**
-   * Returns all InputTableConfig objects associated with this job.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @return all of the table query configs for the job
-   * @since 1.6.0
-   */
-  public static Map<String,org.apache.accumulo.core.client.mapreduce.InputTableConfig>
-      getInputTableConfigs(Class<?> implementingClass, Configuration conf) {
-    Map<String,org.apache.accumulo.core.client.mapreduce.InputTableConfig> configs =
-        new HashMap<>();
-    Map.Entry<String,org.apache.accumulo.core.client.mapreduce.InputTableConfig> defaultConfig =
-        getDefaultInputTableConfig(implementingClass, conf);
-    if (defaultConfig != null) {
-      configs.put(defaultConfig.getKey(), defaultConfig.getValue());
-    }
-    String configString = conf.get(enumToConfKey(implementingClass, ScanOpts.TABLE_CONFIGS));
-    MapWritable mapWritable = new MapWritable();
-    if (configString != null) {
-      try {
-        byte[] bytes = Base64.getDecoder().decode(configString);
-        ByteArrayInputStream bais = new ByteArrayInputStream(bytes);
-        mapWritable.readFields(new DataInputStream(bais));
-        bais.close();
-      } catch (IOException e) {
-        throw new IllegalStateException("The table query configurations could not be deserialized"
-            + " from the given configuration");
-      }
-    }
-    for (Map.Entry<Writable,Writable> entry : mapWritable.entrySet()) {
-      configs.put(entry.getKey().toString(),
-          (org.apache.accumulo.core.client.mapreduce.InputTableConfig) entry.getValue());
-    }
-
-    return configs;
-  }
-
-  /**
-   * Returns the InputTableConfig for the given table
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @param tableName the table name for which to fetch the table query config
-   * @return the table query config for the given table name (if it exists) and null if it does not
-   * @since 1.6.0
-   */
-  public static org.apache.accumulo.core.client.mapreduce.InputTableConfig
-      getInputTableConfig(Class<?> implementingClass, Configuration conf, String tableName) {
-    Map<String,org.apache.accumulo.core.client.mapreduce.InputTableConfig> queryConfigs =
-        getInputTableConfigs(implementingClass, conf);
-    return queryConfigs.get(tableName);
-  }
-
-  private static String extractNamespace(final String tableName) {
-    final int delimiterPos = tableName.indexOf('.');
-    if (delimiterPos < 1) {
-      return ""; // default namespace
-    } else {
-      return tableName.substring(0, delimiterPos);
-    }
-  }
-
-  /**
-   * Validates that the user has permissions on the requested tables
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @since 1.7.0
-   */
-  public static void validatePermissions(Class<?> implementingClass, Configuration conf)
-      throws IOException {
-    Map<String,org.apache.accumulo.core.client.mapreduce.InputTableConfig> inputTableConfigs =
-        getInputTableConfigs(implementingClass, conf);
-    try {
-      AccumuloClient client = client(implementingClass, conf);
-      if (getInputTableConfigs(implementingClass, conf).isEmpty()) {
-        throw new IOException("No table set.");
-      }
-
-      String principal = getPrincipal(implementingClass, conf);
-      if (principal == null) {
-        principal = client.whoami();
-      }
-
-      for (Map.Entry<String,
-          org.apache.accumulo.core.client.mapreduce.InputTableConfig> tableConfig : inputTableConfigs
-              .entrySet()) {
-        final String tableName = tableConfig.getKey();
-        final String namespace = extractNamespace(tableName);
-        final boolean hasTableRead = client.securityOperations().hasTablePermission(principal,
-            tableName, TablePermission.READ);
-        final boolean hasNamespaceRead = client.securityOperations()
-            .hasNamespacePermission(principal, namespace, NamespacePermission.READ);
-        if (!hasTableRead && !hasNamespaceRead) {
-          throw new IOException("Unable to access table");
-        }
-      }
-      for (Map.Entry<String,
-          org.apache.accumulo.core.client.mapreduce.InputTableConfig> tableConfigEntry : inputTableConfigs
-              .entrySet()) {
-        org.apache.accumulo.core.client.mapreduce.InputTableConfig tableConfig =
-            tableConfigEntry.getValue();
-        if (!tableConfig.shouldUseLocalIterators()) {
-          if (tableConfig.getIterators() != null) {
-            for (IteratorSetting iter : tableConfig.getIterators()) {
-              if (!client.tableOperations().testClassLoad(tableConfigEntry.getKey(),
-                  iter.getIteratorClass(), SortedKeyValueIterator.class.getName())) {
-                throw new AccumuloException("Servers are unable to load " + iter.getIteratorClass()
-                    + " as a " + SortedKeyValueIterator.class.getName());
-              }
-            }
-          }
-        }
-      }
-    } catch (AccumuloException | TableNotFoundException | AccumuloSecurityException e) {
-      throw new IOException(e);
-    }
-  }
-
-  /**
-   * Returns the InputTableConfig for the configuration based on the properties set using the
-   * single-table input methods.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop instance for which to retrieve the configuration
-   * @return the config object built from the single input table properties set on the job
-   * @since 1.6.0
-   */
-  protected static Map.Entry<String,org.apache.accumulo.core.client.mapreduce.InputTableConfig>
-      getDefaultInputTableConfig(Class<?> implementingClass, Configuration conf) {
-    String tableName = getInputTableName(implementingClass, conf);
-    if (tableName != null) {
-      org.apache.accumulo.core.client.mapreduce.InputTableConfig queryConfig =
-          new org.apache.accumulo.core.client.mapreduce.InputTableConfig();
-      List<IteratorSetting> itrs = getIterators(implementingClass, conf);
-      if (itrs != null) {
-        queryConfig.setIterators(itrs);
-      }
-      Set<Pair<Text,Text>> columns = getFetchedColumns(implementingClass, conf);
-      if (columns != null) {
-        queryConfig.fetchColumns(columns);
-      }
-      List<Range> ranges = null;
-      try {
-        ranges = getRanges(implementingClass, conf);
-      } catch (IOException e) {
-        throw new RuntimeException(e);
-      }
-      if (ranges != null) {
-        queryConfig.setRanges(ranges);
-      }
-
-      SamplerConfiguration samplerConfig = getSamplerConfiguration(implementingClass, conf);
-      if (samplerConfig != null) {
-        queryConfig.setSamplerConfiguration(samplerConfig);
-      }
-
-      queryConfig.setAutoAdjustRanges(getAutoAdjustRanges(implementingClass, conf))
-          .setUseIsolatedScanners(isIsolated(implementingClass, conf))
-          .setUseLocalIterators(usesLocalIterators(implementingClass, conf))
-          .setOfflineScan(isOfflineScan(implementingClass, conf));
-      return Maps.immutableEntry(tableName, queryConfig);
-    }
-    return null;
-  }
-
-  public static Map<String,Map<KeyExtent,List<Range>>> binOffline(TableId tableId,
-      List<Range> ranges, ClientContext context) throws AccumuloException, TableNotFoundException {
-
-    Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<>();
-
-    if (context.getTableState(tableId) != TableState.OFFLINE) {
-      context.clearTableListCache();
-      if (context.getTableState(tableId) != TableState.OFFLINE) {
-        throw new AccumuloException(
-            "Table is online tableId:" + tableId + " cannot scan table in offline mode ");
-      }
-    }
-
-    for (Range range : ranges) {
-      Text startRow;
-
-      if (range.getStartKey() != null) {
-        startRow = range.getStartKey().getRow();
-      } else {
-        startRow = new Text();
-      }
-
-      Range metadataRange =
-          new Range(new KeyExtent(tableId, startRow, null).toMetaRow(), true, null, false);
-      Scanner scanner = context.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
-      TabletColumnFamily.PREV_ROW_COLUMN.fetch(scanner);
-      scanner.fetchColumnFamily(LastLocationColumnFamily.NAME);
-      scanner.fetchColumnFamily(CurrentLocationColumnFamily.NAME);
-      scanner.fetchColumnFamily(FutureLocationColumnFamily.NAME);
-      scanner.setRange(metadataRange);
-
-      RowIterator rowIter = new RowIterator(scanner);
-      KeyExtent lastExtent = null;
-      while (rowIter.hasNext()) {
-        Iterator<Map.Entry<Key,Value>> row = rowIter.next();
-        String last = "";
-        KeyExtent extent = null;
-        String location = null;
-
-        while (row.hasNext()) {
-          Map.Entry<Key,Value> entry = row.next();
-          Key key = entry.getKey();
-
-          if (key.getColumnFamily().equals(LastLocationColumnFamily.NAME)) {
-            last = entry.getValue().toString();
-          }
-
-          if (key.getColumnFamily().equals(CurrentLocationColumnFamily.NAME)
-              || key.getColumnFamily().equals(FutureLocationColumnFamily.NAME)) {
-            location = entry.getValue().toString();
-          }
-
-          if (TabletColumnFamily.PREV_ROW_COLUMN.hasColumns(key)) {
-            extent = KeyExtent.fromMetaPrevRow(entry);
-          }
-
-        }
-
-        if (location != null) {
-          return null;
-        }
-
-        if (!extent.tableId().equals(tableId)) {
-          throw new AccumuloException("Saw unexpected table Id " + tableId + " " + extent);
-        }
-
-        if (lastExtent != null && !extent.isPreviousExtent(lastExtent)) {
-          throw new AccumuloException(" " + lastExtent + " is not previous extent " + extent);
-        }
-
-        binnedRanges.computeIfAbsent(last, k -> new HashMap<>())
-            .computeIfAbsent(extent, k -> new ArrayList<>()).add(range);
-
-        if (extent.endRow() == null
-            || range.afterEndKey(new Key(extent.endRow()).followingKey(PartialKey.ROW))) {
-          break;
-        }
-
-        lastExtent = extent;
-      }
-
-    }
-    return binnedRanges;
-  }
-
-  private static String toBase64(Writable writable) {
-    ByteArrayOutputStream baos = new ByteArrayOutputStream();
-    DataOutputStream dos = new DataOutputStream(baos);
-    try {
-      writable.write(dos);
-      dos.close();
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-
-    return Base64.getEncoder().encodeToString(baos.toByteArray());
-  }
-
-  private static <T extends Writable> T fromBase64(T writable, String enc) {
-    ByteArrayInputStream bais = new ByteArrayInputStream(Base64.getDecoder().decode(enc));
-    DataInputStream dis = new DataInputStream(bais);
-    try {
-      writable.readFields(dis);
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-    return writable;
-  }
-
-  public static void setSamplerConfiguration(Class<?> implementingClass, Configuration conf,
-      SamplerConfiguration samplerConfig) {
-    requireNonNull(samplerConfig);
-
-    String key = enumToConfKey(implementingClass, ScanOpts.SAMPLER_CONFIG);
-    String val = toBase64(new SamplerConfigurationImpl(samplerConfig));
-
-    conf.set(key, val);
-  }
-
-  private static SamplerConfiguration getSamplerConfiguration(Class<?> implementingClass,
-      Configuration conf) {
-    String key = enumToConfKey(implementingClass, ScanOpts.SAMPLER_CONFIG);
-
-    String encodedSC = conf.get(key);
-    if (encodedSC == null) {
-      return null;
-    }
-
-    return fromBase64(new SamplerConfigurationImpl(), encodedSC).toSamplerConfiguration();
-  }
-
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/OutputConfigurator.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/OutputConfigurator.java
deleted file mode 100644
index e557131..0000000
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/OutputConfigurator.java
+++ /dev/null
@@ -1,203 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.clientImpl.mapreduce.lib;
-
-import static java.nio.charset.StandardCharsets.UTF_8;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.BatchWriterConfig;
-import org.apache.hadoop.conf.Configuration;
-
-public class OutputConfigurator extends ConfiguratorBase {
-
-  /**
-   * Configuration keys for {@link BatchWriter}.
-   *
-   * @since 1.6.0
-   */
-  public static enum WriteOpts {
-    DEFAULT_TABLE_NAME, BATCH_WRITER_CONFIG
-  }
-
-  /**
-   * Configuration keys for various features.
-   *
-   * @since 1.6.0
-   */
-  public static enum Features {
-    CAN_CREATE_TABLES, SIMULATION_MODE
-  }
-
-  /**
-   * Sets the default table name to use if one emits a null in place of a table name for a given
-   * mutation. Table names can only be alpha-numeric and underscores.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @param tableName the table to use when the tablename is null in the write call
-   * @since 1.6.0
-   */
-  public static void setDefaultTableName(Class<?> implementingClass, Configuration conf,
-      String tableName) {
-    if (tableName != null) {
-      conf.set(enumToConfKey(implementingClass, WriteOpts.DEFAULT_TABLE_NAME), tableName);
-    }
-  }
-
-  /**
-   * Gets the default table name from the configuration.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @return the default table name
-   * @since 1.6.0
-   * @see #setDefaultTableName(Class, Configuration, String)
-   */
-  public static String getDefaultTableName(Class<?> implementingClass, Configuration conf) {
-    return conf.get(enumToConfKey(implementingClass, WriteOpts.DEFAULT_TABLE_NAME));
-  }
-
-  /**
-   * Sets the configuration for for the job's {@link BatchWriter} instances. If not set, a new
-   * {@link BatchWriterConfig}, with sensible built-in defaults is used. Setting the configuration
-   * multiple times overwrites any previous configuration.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @param bwConfig the configuration for the {@link BatchWriter}
-   * @since 1.6.0
-   */
-  public static void setBatchWriterOptions(Class<?> implementingClass, Configuration conf,
-      BatchWriterConfig bwConfig) {
-    ByteArrayOutputStream baos = new ByteArrayOutputStream();
-    String serialized;
-    try {
-      bwConfig.write(new DataOutputStream(baos));
-      serialized = new String(baos.toByteArray(), UTF_8);
-      baos.close();
-    } catch (IOException e) {
-      throw new IllegalArgumentException(
-          "unable to serialize " + BatchWriterConfig.class.getName());
-    }
-    conf.set(enumToConfKey(implementingClass, WriteOpts.BATCH_WRITER_CONFIG), serialized);
-  }
-
-  /**
-   * Gets the {@link BatchWriterConfig} settings.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @return the configuration object
-   * @since 1.6.0
-   * @see #setBatchWriterOptions(Class, Configuration, BatchWriterConfig)
-   */
-  public static BatchWriterConfig getBatchWriterOptions(Class<?> implementingClass,
-      Configuration conf) {
-    String serialized = conf.get(enumToConfKey(implementingClass, WriteOpts.BATCH_WRITER_CONFIG));
-    BatchWriterConfig bwConfig = new BatchWriterConfig();
-    if (serialized == null || serialized.isEmpty()) {
-      return bwConfig;
-    } else {
-      try {
-        ByteArrayInputStream bais = new ByteArrayInputStream(serialized.getBytes(UTF_8));
-        bwConfig.readFields(new DataInputStream(bais));
-        bais.close();
-        return bwConfig;
-      } catch (IOException e) {
-        throw new IllegalArgumentException(
-            "unable to serialize " + BatchWriterConfig.class.getName());
-      }
-    }
-  }
-
-  /**
-   * Sets the directive to create new tables, as necessary. Table names can only be alpha-numeric
-   * and underscores.
-   *
-   * <p>
-   * By default, this feature is <b>disabled</b>.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @param enableFeature the feature is enabled if true, disabled otherwise
-   * @since 1.6.0
-   */
-  public static void setCreateTables(Class<?> implementingClass, Configuration conf,
-      boolean enableFeature) {
-    conf.setBoolean(enumToConfKey(implementingClass, Features.CAN_CREATE_TABLES), enableFeature);
-  }
-
-  /**
-   * Determines whether tables are permitted to be created as needed.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @return true if the feature is disabled, false otherwise
-   * @since 1.6.0
-   * @see #setCreateTables(Class, Configuration, boolean)
-   */
-  public static Boolean canCreateTables(Class<?> implementingClass, Configuration conf) {
-    return conf.getBoolean(enumToConfKey(implementingClass, Features.CAN_CREATE_TABLES), false);
-  }
-
-  /**
-   * Sets the directive to use simulation mode for this job. In simulation mode, no output is
-   * produced. This is useful for testing.
-   *
-   * <p>
-   * By default, this feature is <b>disabled</b>.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @param enableFeature the feature is enabled if true, disabled otherwise
-   * @since 1.6.0
-   */
-  public static void setSimulationMode(Class<?> implementingClass, Configuration conf,
-      boolean enableFeature) {
-    conf.setBoolean(enumToConfKey(implementingClass, Features.SIMULATION_MODE), enableFeature);
-  }
-
-  /**
-   * Determines whether this feature is enabled.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @return true if the feature is enabled, false otherwise
-   * @since 1.6.0
-   * @see #setSimulationMode(Class, Configuration, boolean)
-   */
-  public static Boolean getSimulationMode(Class<?> implementingClass, Configuration conf) {
-    return conf.getBoolean(enumToConfKey(implementingClass, Features.SIMULATION_MODE), false);
-  }
-
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/compaction/ShellCompactCommandConfigurer.java b/core/src/main/java/org/apache/accumulo/core/compaction/ShellCompactCommandConfigurer.java
new file mode 100644
index 0000000..6efa34a
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/compaction/ShellCompactCommandConfigurer.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.compaction;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+
+import org.apache.accumulo.core.client.admin.compaction.CompactionConfigurer;
+import org.apache.accumulo.core.conf.Property;
+
+/**
+ * The compaction configurer is used by the shell compact command. It exists in accumulo-core, so it
+ * is on the class path for the shell and servers that run compactions.
+ */
+public class ShellCompactCommandConfigurer implements CompactionConfigurer {
+
+  private Map<String,String> overrides = new HashMap<>();
+
+  @Override
+  public void init(InitParameters iparams) {
+    Set<Entry<String,String>> es = iparams.getOptions().entrySet();
+    for (Entry<String,String> entry : es) {
+
+      switch (CompactionSettings.valueOf(entry.getKey())) {
+        case OUTPUT_COMPRESSION_OPT:
+          overrides.put(Property.TABLE_FILE_COMPRESSION_TYPE.getKey(), entry.getValue());
+          break;
+        case OUTPUT_BLOCK_SIZE_OPT:
+          overrides.put(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE.getKey(), entry.getValue());
+          break;
+        case OUTPUT_INDEX_BLOCK_SIZE_OPT:
+          overrides.put(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE_INDEX.getKey(), entry.getValue());
+          break;
+        case OUTPUT_HDFS_BLOCK_SIZE_OPT:
+          overrides.put(Property.TABLE_FILE_BLOCK_SIZE.getKey(), entry.getValue());
+          break;
+        case OUTPUT_REPLICATION_OPT:
+          overrides.put(Property.TABLE_FILE_REPLICATION.getKey(), entry.getValue());
+          break;
+        default:
+          throw new IllegalArgumentException("Unknown option " + entry.getKey());
+      }
+    }
+  }
+
+  @Override
+  public Overrides override(InputParameters params) {
+    return new Overrides(overrides);
+  }
+
+}
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/strategies/ConfigurableCompactionStrategy.java b/core/src/main/java/org/apache/accumulo/core/compaction/ShellCompactCommandSelector.java
similarity index 80%
rename from server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/strategies/ConfigurableCompactionStrategy.java
rename to core/src/main/java/org/apache/accumulo/core/compaction/ShellCompactCommandSelector.java
index 6ab51d4..90d5413 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/strategies/ConfigurableCompactionStrategy.java
+++ b/core/src/main/java/org/apache/accumulo/core/compaction/ShellCompactCommandSelector.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.accumulo.tserver.compaction.strategies;
+package org.apache.accumulo.core.compaction;
 
 import java.util.ArrayList;
 import java.util.Collection;
@@ -32,13 +32,10 @@
 import java.util.stream.Collectors;
 
 import org.apache.accumulo.core.client.admin.compaction.CompactableFile;
-import org.apache.accumulo.core.client.admin.compaction.CompactionConfigurer;
 import org.apache.accumulo.core.client.admin.compaction.CompactionSelector;
 import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
 import org.apache.accumulo.core.client.summary.Summary;
-import org.apache.accumulo.core.compaction.CompactionSettings;
 import org.apache.accumulo.core.conf.ConfigurationCopy;
-import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
@@ -46,9 +43,10 @@
 import org.apache.hadoop.fs.Path;
 
 /**
- * The compaction strategy used by the shell compact command.
+ * The compaction selector is used by the shell compact command. It exists in accumulo-core, so it
+ * is on the class path for the shell and servers that run compactions.
  */
-public class ConfigurableCompactionStrategy implements CompactionSelector, CompactionConfigurer {
+public class ShellCompactCommandSelector implements CompactionSelector {
 
   private abstract static class Test {
     abstract Set<CompactableFile> getFilesToCompact(SelectionParameters params);
@@ -168,45 +166,9 @@
   private List<Test> tests = new ArrayList<>();
   private boolean andTest = true;
   private int minFiles = 1;
-  private Map<String,String> overrides = new HashMap<>();
 
   @Override
-  public void init(
-      org.apache.accumulo.core.client.admin.compaction.CompactionConfigurer.InitParameters iparams) {
-    Set<Entry<String,String>> es = iparams.getOptions().entrySet();
-    for (Entry<String,String> entry : es) {
-
-      switch (CompactionSettings.valueOf(entry.getKey())) {
-        case OUTPUT_COMPRESSION_OPT:
-          overrides.put(Property.TABLE_FILE_COMPRESSION_TYPE.getKey(), entry.getValue());
-          break;
-        case OUTPUT_BLOCK_SIZE_OPT:
-          overrides.put(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE.getKey(), entry.getValue());
-          break;
-        case OUTPUT_INDEX_BLOCK_SIZE_OPT:
-          overrides.put(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE_INDEX.getKey(), entry.getValue());
-          break;
-        case OUTPUT_HDFS_BLOCK_SIZE_OPT:
-          overrides.put(Property.TABLE_FILE_BLOCK_SIZE.getKey(), entry.getValue());
-          break;
-        case OUTPUT_REPLICATION_OPT:
-          overrides.put(Property.TABLE_FILE_REPLICATION.getKey(), entry.getValue());
-          break;
-        default:
-          throw new IllegalArgumentException("Unknown option " + entry.getKey());
-      }
-    }
-
-  }
-
-  @Override
-  public Overrides override(InputParameters params) {
-    return new Overrides(overrides);
-  }
-
-  @Override
-  public void init(
-      org.apache.accumulo.core.client.admin.compaction.CompactionSelector.InitParameters iparams) {
+  public void init(InitParameters iparams) {
     boolean selectNoSummary = false;
     boolean selectExtraSummary = false;
 
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/AccumuloConfiguration.java b/core/src/main/java/org/apache/accumulo/core/conf/AccumuloConfiguration.java
index 636b18d..8a6e48b 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/AccumuloConfiguration.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/AccumuloConfiguration.java
@@ -45,7 +45,6 @@
 import java.util.stream.StreamSupport;
 
 import org.apache.accumulo.core.conf.PropertyType.PortRange;
-import org.apache.accumulo.core.spi.scan.SimpleScanDispatcher;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -411,11 +410,6 @@
      * Re-reads the max threads from the configuration that created this class
      */
     public int getCurrentMaxThreads() {
-      Integer depThreads = getDeprecatedScanThreads(name, isScanServer);
-      if (depThreads != null) {
-        return depThreads;
-      }
-
       if (isScanServer) {
         String prop =
             Property.SSERV_SCAN_EXECUTORS_PREFIX.getKey() + name + "." + SCAN_EXEC_THREADS;
@@ -436,43 +430,6 @@
    */
   public abstract boolean isPropertySet(Property prop);
 
-  // deprecation property warning could get spammy in tserver so only warn once
-  boolean depPropWarned = false;
-
-  @SuppressWarnings("deprecation")
-  Integer getDeprecatedScanThreads(String name, boolean isScanServer) {
-
-    Property prop;
-    Property deprecatedProp;
-
-    if (name.equals(SimpleScanDispatcher.DEFAULT_SCAN_EXECUTOR_NAME)) {
-      prop = isScanServer ? Property.SSERV_SCAN_EXECUTORS_DEFAULT_THREADS
-          : Property.TSERV_SCAN_EXECUTORS_DEFAULT_THREADS;
-      deprecatedProp = Property.TSERV_READ_AHEAD_MAXCONCURRENT;
-    } else if (name.equals("meta")) {
-      prop = isScanServer ? Property.SSERV_SCAN_EXECUTORS_META_THREADS
-          : Property.TSERV_SCAN_EXECUTORS_META_THREADS;
-      deprecatedProp = Property.TSERV_METADATA_READ_AHEAD_MAXCONCURRENT;
-    } else {
-      return null;
-    }
-
-    if (!isPropertySet(prop) && isPropertySet(deprecatedProp)) {
-      if (!depPropWarned) {
-        depPropWarned = true;
-        log.warn("Property {} is deprecated, use {} instead.", deprecatedProp.getKey(),
-            prop.getKey());
-      }
-      return Integer.valueOf(get(deprecatedProp));
-    } else if (isPropertySet(prop) && isPropertySet(deprecatedProp) && !depPropWarned) {
-      depPropWarned = true;
-      log.warn("Deprecated property {} ignored because {} is set", deprecatedProp.getKey(),
-          prop.getKey());
-    }
-
-    return null;
-  }
-
   private static class RefCount<T> {
     T obj;
     long count;
@@ -591,12 +548,7 @@
         String val = subEntry.getValue();
 
         if (opt.equals(SCAN_EXEC_THREADS)) {
-          Integer depThreads = getDeprecatedScanThreads(name, isScanServer);
-          if (depThreads == null) {
-            threads = Integer.parseInt(val);
-          } else {
-            threads = depThreads;
-          }
+          threads = Integer.parseInt(val);
         } else if (opt.equals(SCAN_EXEC_PRIORITY)) {
           prio = Integer.parseInt(val);
         } else if (opt.equals(SCAN_EXEC_PRIORITIZER)) {
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/ClientConfigGenerate.java b/core/src/main/java/org/apache/accumulo/core/conf/ClientConfigGenerate.java
index 0cd2175..f95688a 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/ClientConfigGenerate.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/ClientConfigGenerate.java
@@ -29,8 +29,8 @@
 import com.google.common.collect.Sets;
 
 /**
- * Generates client-properties.md for documentation on Accumulo website and
- * accumulo-client.properties for Accumulo distribution tarball
+ * Generates client properties documentation on Accumulo website and accumulo-client.properties for
+ * Accumulo distribution tarball
  */
 public class ClientConfigGenerate {
 
@@ -86,9 +86,9 @@
     @Override
     void pageHeader() {
       doc.println("---");
-      doc.println("title: Client Properties (2.x)");
+      doc.println("title: Client Properties (3.x)");
       doc.println("category: configuration");
-      doc.println("order: 3");
+      doc.println("order: 4");
       doc.println("---\n");
       doc.println("<!-- WARNING: Do not edit this file. It is a generated file"
           + " that is copied from Accumulo build (from core/target/generated-docs) -->");
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/ClientProperty.java b/core/src/main/java/org/apache/accumulo/core/conf/ClientProperty.java
index 34780c8..39c3e75 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/ClientProperty.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/ClientProperty.java
@@ -28,7 +28,6 @@
 import java.util.Objects;
 import java.util.Properties;
 
-import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.admin.TableOperations.ImportMappingOptions;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.client.security.tokens.CredentialProviderToken;
@@ -135,16 +134,7 @@
       "The maximum duration to leave idle transports open in the client's transport pool", "2.1.0",
       false),
 
-  // Trace
-  @Deprecated(since = "2.1.0", forRemoval = true)
-  TRACE_SPAN_RECEIVERS("trace.span.receivers", "org.apache.accumulo.tracer.ZooTraceClient",
-      "A list of span receiver classes to send trace spans"),
-  @Deprecated(since = "2.1.0", forRemoval = true)
-  TRACE_ZOOKEEPER_PATH("trace.zookeeper.path", Constants.ZTRACERS, PropertyType.PATH,
-      "The zookeeper node where tracers are registered", "2.0.0", false);
-
-  @Deprecated(since = "2.1.0", forRemoval = true)
-  public static final String TRACE_SPAN_RECEIVER_PREFIX = "trace.span.receiver";
+  ;
 
   private final String key;
   private final String defaultValue;
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/ConfigCheckUtil.java b/core/src/main/java/org/apache/accumulo/core/conf/ConfigCheckUtil.java
index eba5cc4..fcb255c 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/ConfigCheckUtil.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/ConfigCheckUtil.java
@@ -18,7 +18,6 @@
  */
 package org.apache.accumulo.core.conf;
 
-import java.io.IOException;
 import java.util.Map.Entry;
 import java.util.Objects;
 
@@ -44,6 +43,7 @@
    * {@link Property#INSTANCE_ZK_TIMEOUT} within a valid range.
    *
    * @param entries iterable through configuration keys and values
+   * @param source the namespace, table id, site or system config where for diagnostic messages
    * @throws ConfigCheckException if a fatal configuration error is found
    */
   public static void validate(Iterable<Entry<String,String>> entries, @NonNull String source) {
@@ -160,7 +160,7 @@
       Class<?> requiredBaseClass) {
     try {
       ConfigurationTypeHelper.getClassInstance(null, className, requiredBaseClass);
-    } catch (IOException | ReflectiveOperationException e) {
+    } catch (ReflectiveOperationException e) {
       fatal(confOption + " has an invalid class name: " + className);
     } catch (ClassCastException e) {
       fatal(confOption + " must implement " + requiredBaseClass
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/ConfigurationDocGen.java b/core/src/main/java/org/apache/accumulo/core/conf/ConfigurationDocGen.java
index 51a6e24..dc5320e 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/ConfigurationDocGen.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/ConfigurationDocGen.java
@@ -62,9 +62,9 @@
 
   void pageHeader() {
     doc.println("---");
-    doc.println("title: Server Properties (2.x)");
+    doc.println("title: Server Properties (3.x)");
     doc.println("category: configuration");
-    doc.println("order: 5");
+    doc.println("order: 6");
     doc.println("---\n");
     doc.println("<!-- WARNING: Do not edit this file. It is a generated file"
         + " that is copied from Accumulo build (from core/target/generated-docs) -->\n");
@@ -100,8 +100,8 @@
     description += "**Available since:** ";
     if (prop.getKey().startsWith("manager.")
         && (prop.availableSince().startsWith("1.") || prop.availableSince().startsWith("2.0"))) {
-      description += "2.1.0 (since " + prop.availableSince() + " as *master."
-          + prop.getKey().substring(8) + "*)<br>";
+      description += "2.1.0 (formerly *master." + prop.getKey().substring(8) + "* since "
+          + prop.availableSince() + ")<br>";
     } else {
       description += prop.availableSince() + "<br>";
     }
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/ConfigurationTypeHelper.java b/core/src/main/java/org/apache/accumulo/core/conf/ConfigurationTypeHelper.java
index 8893ae8..ef5e0b3 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/ConfigurationTypeHelper.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/ConfigurationTypeHelper.java
@@ -24,7 +24,6 @@
 import static java.util.concurrent.TimeUnit.MINUTES;
 import static java.util.concurrent.TimeUnit.SECONDS;
 
-import java.io.IOException;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
@@ -173,7 +172,7 @@
 
     try {
       instance = getClassInstance(context, clazzName, base);
-    } catch (RuntimeException | IOException | ReflectiveOperationException e) {
+    } catch (RuntimeException | ReflectiveOperationException e) {
       log.error("Failed to load class {} in classloader context {}", clazzName, context, e);
     }
 
@@ -193,7 +192,7 @@
    * @return a new instance of the class
    */
   public static <T> T getClassInstance(String context, String clazzName, Class<T> base)
-      throws IOException, ReflectiveOperationException {
+      throws ReflectiveOperationException {
     T instance;
 
     Class<? extends T> clazz = ClassLoaderUtil.loadClass(context, clazzName, base);
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/DeprecatedPropertyUtil.java b/core/src/main/java/org/apache/accumulo/core/conf/DeprecatedPropertyUtil.java
index 1152103..8cea352 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/DeprecatedPropertyUtil.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/DeprecatedPropertyUtil.java
@@ -22,14 +22,10 @@
 
 import java.util.ArrayList;
 import java.util.List;
-import java.util.Spliterator;
-import java.util.Spliterators;
 import java.util.function.BiConsumer;
 import java.util.function.Predicate;
 import java.util.function.UnaryOperator;
-import java.util.stream.StreamSupport;
 
-import org.apache.commons.configuration2.AbstractConfiguration;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -52,15 +48,10 @@
 
   private static final Logger log = LoggerFactory.getLogger(DeprecatedPropertyUtil.class);
 
-  @SuppressWarnings("deprecation")
-  public static final PropertyRenamer MASTER_MANAGER_RENAMER = PropertyRenamer
-      .renamePrefix(Property.MASTER_PREFIX.getKey(), Property.MANAGER_PREFIX.getKey());
-
   /**
    * Ordered list of renamers
    */
-  protected static final List<PropertyRenamer> renamers =
-      new ArrayList<>(List.of(MASTER_MANAGER_RENAMER));
+  protected static final List<PropertyRenamer> renamers = new ArrayList<>();
 
   /**
    * Checks if {@code propertyName} is a deprecated property name and return its replacement name,
@@ -97,22 +88,10 @@
   }
 
   /**
-   * Ensures that for any deprecated properties, both the deprecated and replacement property name
-   * are not both used in {@code config}.
-   *
-   * @param config the configuration to check for invalid use of deprecated and replacement
-   *        properties
+   * @return The list of property renamers
    */
-  static void sanityCheckManagerProperties(AbstractConfiguration config) {
-    boolean foundMasterPrefix = StreamSupport
-        .stream(Spliterators.spliteratorUnknownSize(config.getKeys(), Spliterator.ORDERED), false)
-        .anyMatch(MASTER_MANAGER_RENAMER.keyFilter);
-    boolean foundManagerPrefix = StreamSupport
-        .stream(Spliterators.spliteratorUnknownSize(config.getKeys(), Spliterator.ORDERED), false)
-        .anyMatch(k -> k.startsWith(Property.MANAGER_PREFIX.getKey()));
-    if (foundMasterPrefix && foundManagerPrefix) {
-      throw new IllegalStateException("Found both old 'master.*' and new 'manager.*' "
-          + "naming conventions in the same startup configuration");
-    }
+  public static List<PropertyRenamer> getPropertyRenamers() {
+    return renamers;
   }
+
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/Property.java b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
index 6295850..ba65025 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/Property.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
@@ -27,19 +27,18 @@
 import java.util.HashSet;
 import java.util.function.Predicate;
 
-import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.classloader.ClassLoaderUtil;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.data.constraints.NoDeleteConstraint;
 import org.apache.accumulo.core.file.rfile.RFile;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 import org.apache.accumulo.core.iteratorsImpl.system.DeletingIterator;
-import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.spi.compaction.DefaultCompactionPlanner;
 import org.apache.accumulo.core.spi.compaction.SimpleCompactionDispatcher;
 import org.apache.accumulo.core.spi.fs.RandomVolumeChooser;
 import org.apache.accumulo.core.spi.scan.ScanDispatcher;
 import org.apache.accumulo.core.spi.scan.ScanPrioritizer;
+import org.apache.accumulo.core.spi.scan.ScanServerSelector;
 import org.apache.accumulo.core.spi.scan.SimpleScanDispatcher;
 import org.apache.accumulo.core.util.format.DefaultFormatter;
 import org.slf4j.LoggerFactory;
@@ -47,6 +46,23 @@
 import com.google.common.base.Preconditions;
 
 public enum Property {
+  COMPACTION_PREFIX("compaction.", null, PropertyType.PREFIX,
+      "Both major and minor compaction properties can be included under this prefix.", "3.1.0"),
+  COMPACTION_SERVICE_PREFIX(COMPACTION_PREFIX + "service.", null, PropertyType.PREFIX,
+      "This prefix should be used to define all properties for the compaction services."
+          + "See {% jlink -f org.apache.accumulo.core.spi.compaction.DefaultCompactionPlanner %}.\n"
+          + "A new external compaction service would be defined like the following:\n"
+          + "`compaction.service.newService.planner="
+          + "\"org.apache.accumulo.core.spi.compaction.DefaultCompactionPlanner\".`\n"
+          + "`compaction.service.newService.opts.queues=\""
+          + "[{\"name\": \"small\", \"maxSize\":\"32M\"},"
+          + "{ \"name\":\"medium\", \"maxSize\":\"512M\"},{\"name\":\"large\"}]`\n"
+          + "`compaction.service.newService.opts.maxOpen=50`.\n"
+          + "Additional options can be defined using the `compaction.service.<service>.opts.<option>` property.",
+      "3.1.0"),
+  COMPACTION_WARN_TIME(COMPACTION_PREFIX + "warn.time", "10m", PropertyType.TIMEDURATION,
+      "When a compaction has not made progress for this time period, a warning will be logged.",
+      "3.1.0"),
   // SSL properties local to each node (see also instance.ssl.enabled which must be consistent
   // across all nodes in an instance)
   RPC_PREFIX("rpc.", null, PropertyType.PREFIX,
@@ -80,9 +96,9 @@
       "1.6.0"),
   RPC_SSL_CIPHER_SUITES("rpc.ssl.cipher.suites", "", PropertyType.STRING,
       "Comma separated list of cipher suites that can be used by accepted connections.", "1.6.1"),
-  RPC_SSL_ENABLED_PROTOCOLS("rpc.ssl.server.enabled.protocols", "TLSv1.2", PropertyType.STRING,
+  RPC_SSL_ENABLED_PROTOCOLS("rpc.ssl.server.enabled.protocols", "TLSv1.3", PropertyType.STRING,
       "Comma separated list of protocols that can be used to accept connections.", "1.6.2"),
-  RPC_SSL_CLIENT_PROTOCOL("rpc.ssl.client.protocol", "TLSv1.2", PropertyType.STRING,
+  RPC_SSL_CLIENT_PROTOCOL("rpc.ssl.client.protocol", "TLSv1.3", PropertyType.STRING,
       "The protocol used to connect to a secure server. Must be in the list of enabled protocols "
           + "on the server side `rpc.ssl.server.enabled.protocols`.",
       "1.6.2"),
@@ -208,26 +224,12 @@
           + "encryption, replace this classname with an implementation of the"
           + "org.apache.accumulo.core.spi.crypto.CryptoFactory interface.",
       "2.1.0"),
-
   // general properties
   GENERAL_PREFIX("general.", null, PropertyType.PREFIX,
       "Properties in this category affect the behavior of accumulo overall, but"
           + " do not have to be consistent throughout a cloud.",
       "1.3.5"),
-  @Deprecated(since = "2.0.0")
-  GENERAL_DYNAMIC_CLASSPATHS(
-      org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader.DYNAMIC_CLASSPATH_PROPERTY_NAME,
-      org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader.DEFAULT_DYNAMIC_CLASSPATH_VALUE,
-      PropertyType.STRING,
-      "A list of all of the places where changes "
-          + "in jars or classes will force a reload of the classloader. Built-in dynamic class "
-          + "loading will be removed in a future version. If this is needed, consider overriding "
-          + "the Java system class loader with one that has this feature "
-          + "(https://docs.oracle.com/javase/8/docs/api/java/lang/ClassLoader.html#getSystemClassLoader--). "
-          + "Additionally, this property no longer does property interpolation of environment "
-          + "variables, such as '$ACCUMULO_HOME'. Use commons-configuration syntax,"
-          + "'${env:ACCUMULO_HOME}' instead.",
-      "1.3.5"),
+
   GENERAL_CONTEXT_CLASSLOADER_FACTORY("general.context.class.loader.factory", "",
       PropertyType.CLASSNAME,
       "Name of classloader factory to be used to create classloaders for named contexts,"
@@ -268,11 +270,6 @@
       "2.1.0"),
   GENERAL_THREADPOOL_SIZE("general.server.threadpool.size", "1", PropertyType.COUNT,
       "The number of threads to use for server-internal scheduled tasks.", "2.1.0"),
-  @Deprecated(since = "2.1.0")
-  @ReplacedBy(property = GENERAL_THREADPOOL_SIZE)
-  GENERAL_SIMPLETIMER_THREADPOOL_SIZE("general.server.simpletimer.threadpool.size", "1",
-      PropertyType.COUNT, "The number of threads to use for server-internal scheduled tasks.",
-      "1.7.0"),
   // If you update the default type, be sure to update the default used for initialization failures
   // in VolumeManagerImpl
   @Experimental
@@ -297,6 +294,29 @@
   GENERAL_IDLE_PROCESS_INTERVAL("general.metrics.process.idle", "5m", PropertyType.TIMEDURATION,
       "Amount of time a process must be idle before it is considered to be idle by the metrics system.",
       "2.1.3"),
+  GENERAL_LOW_MEM_DETECTOR_INTERVAL("general.low.mem.detector.interval", "5s",
+      PropertyType.TIMEDURATION, "The time interval between low memory checks.", "3.0.0"),
+  GENERAL_LOW_MEM_DETECTOR_THRESHOLD("general.low.mem.detector.threshold", "0.05",
+      PropertyType.FRACTION,
+      "The LowMemoryDetector will report when free memory drops below this percentage of total memory.",
+      "3.0.0"),
+  GENERAL_LOW_MEM_SCAN_PROTECTION("general.low.mem.protection.scan", "false", PropertyType.BOOLEAN,
+      "Scans may be paused or return results early when the server "
+          + "is low on memory and this property is set to true. Enabling this property will incur a slight "
+          + "scan performance penalty when the server is not low on memory.",
+      "3.0.0"),
+  GENERAL_LOW_MEM_MINC_PROTECTION("general.low.mem.protection.compaction.minc", "false",
+      PropertyType.BOOLEAN,
+      "Minor compactions may be paused when the server "
+          + "is low on memory and this property is set to true. Enabling this property will incur a slight "
+          + "compaction performance penalty when the server is not low on memory.",
+      "3.0.0"),
+  GENERAL_LOW_MEM_MAJC_PROTECTION("general.low.mem.protection.compaction.majc", "false",
+      PropertyType.BOOLEAN,
+      "Major compactions may be paused when the server "
+          + "is low on memory and this property is set to true. Enabling this property will incur a slight "
+          + "compaction performance penalty when the server is not low on memory.",
+      "3.0.0"),
   GENERAL_MAX_SCANNER_RETRY_PERIOD("general.max.scanner.retry.period", "5s",
       PropertyType.TIMEDURATION,
       "The maximum amount of time that a Scanner should wait before retrying a failed RPC.",
@@ -314,19 +334,12 @@
           + " was changed and it now can accept multiple class names. The metrics spi was introduced in 2.1.3,"
           + " the deprecated factory is org.apache.accumulo.core.metrics.MeterRegistryFactory.",
       "2.1.0"),
+  GENERAL_PROCESS_BIND_ADDRESS("general.process.bind.addr", "0.0.0.0", PropertyType.STRING,
+      "The local IP address to which this server should bind for sending and receiving network traffic.",
+      "3.0.0"),
   // properties that are specific to manager server behavior
   MANAGER_PREFIX("manager.", null, PropertyType.PREFIX,
-      "Properties in this category affect the behavior of the manager server. "
-          + "Since 2.1.0, all properties in this category replace the old `master.*` names.",
-      "2.1.0"),
-  @Deprecated(since = "2.1.0")
-  @ReplacedBy(property = Property.MANAGER_PREFIX)
-  MASTER_PREFIX("master.", null, PropertyType.PREFIX,
-      "Properties in this category affect the behavior of the manager (formerly named master) server. "
-          + "Since 2.1.0, all properties in this category are deprecated and replaced with corresponding "
-          + "`manager.*` properties. The old `master.*` names can still be used until at release 3.0, but a warning "
-          + "will be emitted. Configuration files should be updated to use the new property names.",
-      "1.3.5"),
+      "Properties in this category affect the behavior of the manager server.", "2.1.0"),
   MANAGER_CLIENTPORT("manager.port.client", "9999", PropertyType.PORT,
       "The port used for handling client connections on the manager.", "1.3.5"),
   MANAGER_TABLET_BALANCER("manager.tablet.balancer",
@@ -338,28 +351,11 @@
       PropertyType.TIMEDURATION,
       "Time to wait between scanning tablet states to identify tablets that need to be assigned, un-assigned, migrated, etc.",
       "2.1.2"),
-  MANAGER_BULK_RETRIES("manager.bulk.retries", "3", PropertyType.COUNT,
-      "The number of attempts to bulk import a RFile before giving up.", "1.4.0"),
-  MANAGER_BULK_THREADPOOL_SIZE("manager.bulk.threadpool.size", "5", PropertyType.COUNT,
-      "The number of threads to use when coordinating a bulk import.", "1.4.0"),
-  MANAGER_BULK_THREADPOOL_TIMEOUT("manager.bulk.threadpool.timeout", "0s",
-      PropertyType.TIMEDURATION,
-      "The time after which bulk import threads terminate with no work available.  Zero (0) will keep the threads alive indefinitely.",
-      "2.1.0"),
   MANAGER_BULK_TIMEOUT("manager.bulk.timeout", "5m", PropertyType.TIMEDURATION,
       "The time to wait for a tablet server to process a bulk import request.", "1.4.3"),
   MANAGER_RENAME_THREADS("manager.rename.threadpool.size", "20", PropertyType.COUNT,
       "The number of threads to use when renaming user files during table import or bulk ingest.",
       "2.1.0"),
-  @Deprecated(since = "2.1.0")
-  @ReplacedBy(property = MANAGER_RENAME_THREADS)
-  MANAGER_BULK_RENAME_THREADS("manager.bulk.rename.threadpool.size", "20", PropertyType.COUNT,
-      "The number of threads to use when moving user files to bulk ingest "
-          + "directories under accumulo control.",
-      "1.7.0"),
-  MANAGER_BULK_TSERVER_REGEX("manager.bulk.tserver.regex", "", PropertyType.STRING,
-      "Regular expression that defines the set of Tablet Servers that will perform bulk imports.",
-      "2.0.0"),
   MANAGER_MINTHREADS("manager.server.threads.minimum", "20", PropertyType.COUNT,
       "The minimum number of threads to use to handle incoming requests.", "1.4.0"),
   MANAGER_MINTHREADS_TIMEOUT("manager.server.threads.timeout", "0s", PropertyType.TIMEDURATION,
@@ -380,11 +376,6 @@
   MANAGER_WAL_CLOSER_IMPLEMENTATION("manager.wal.closer.implementation",
       "org.apache.accumulo.server.manager.recovery.HadoopLogCloser", PropertyType.CLASSNAME,
       "A class that implements a mechanism to steal write access to a write-ahead log.", "2.1.0"),
-  @Deprecated(since = "2.1.0")
-  @ReplacedBy(property = Property.MANAGER_WAL_CLOSER_IMPLEMENTATION)
-  MANAGER_WALOG_CLOSER_IMPLEMETATION("manager.walog.closer.implementation",
-      "org.apache.accumulo.server.manager.recovery.HadoopLogCloser", PropertyType.CLASSNAME,
-      "A class that implements a mechanism to steal write access to a write-ahead log.", "1.5.0"),
   MANAGER_FATE_METRICS_MIN_UPDATE_INTERVAL("manager.fate.metrics.min.update.interval", "60s",
       PropertyType.TIMEDURATION, "Limit calls from metric sinks to zookeeper to update interval.",
       "1.9.3"),
@@ -392,29 +383,12 @@
       "The number of threads used to run fault-tolerant executions (FATE)."
           + " These are primarily table operations like merge.",
       "1.4.3"),
-  @Deprecated(since = "2.1.0")
-  MANAGER_REPLICATION_SCAN_INTERVAL("manager.replication.status.scan.interval", "30s",
-      PropertyType.TIMEDURATION,
-      "Amount of time to sleep before scanning the status section of the "
-          + "replication table for new data.",
-      "1.7.0"),
-  @Deprecated(since = "2.1.0")
-  MANAGER_REPLICATION_COORDINATOR_PORT("manager.replication.coordinator.port", "10001",
-      PropertyType.PORT, "Port for the replication coordinator service.", "1.7.0"),
-  @Deprecated(since = "2.1.0")
-  MANAGER_REPLICATION_COORDINATOR_MINTHREADS("manager.replication.coordinator.minthreads", "4",
-      PropertyType.COUNT, "Minimum number of threads dedicated to answering coordinator requests.",
-      "1.7.0"),
-  @Deprecated(since = "2.1.0")
-  MANAGER_REPLICATION_COORDINATOR_THREADCHECK("manager.replication.coordinator.threadcheck.time",
-      "5s", PropertyType.TIMEDURATION,
-      "The time between adjustments of the coordinator thread pool.", "1.7.0"),
   MANAGER_STATUS_THREAD_POOL_SIZE("manager.status.threadpool.size", "0", PropertyType.COUNT,
       "The number of threads to use when fetching the tablet server status for balancing.  Zero "
           + "indicates an unlimited number of threads will be used.",
       "1.8.0"),
   MANAGER_METADATA_SUSPENDABLE("manager.metadata.suspendable", "false", PropertyType.BOOLEAN,
-      "Allow tablets for the " + MetadataTable.NAME
+      "Allow tablets for the " + AccumuloTable.METADATA.tableName()
           + " table to be suspended via table.suspend.duration.",
       "1.8.0"),
   MANAGER_STARTUP_TSERVER_AVAIL_MIN_COUNT("manager.startup.tserver.avail.min.count", "0",
@@ -431,6 +405,15 @@
           + "indefinitely. Default is 0 to block indefinitely. Only valid when tserver available "
           + "threshold is set greater than 0.",
       "1.10.0"),
+  SPLIT_PREFIX("split.", null, PropertyType.PREFIX,
+      "System wide properties related to splitting tablets.", "3.1.0"),
+  SPLIT_MAXOPEN("split.files.max", "300", PropertyType.COUNT,
+      "To find a tablets split points, all RFiles are opened and their indexes"
+          + " are read. This setting determines how many RFiles can be opened at once."
+          + " When there are more RFiles than this setting multiple passes must be"
+          + " made, which is slower. However opening too many RFiles at once can cause"
+          + " problems.",
+      "3.1.0"),
   // properties that are specific to scan server behavior
   @Experimental
   SSERV_PREFIX("sserver.", null, PropertyType.PREFIX,
@@ -448,6 +431,13 @@
   SSERV_DEFAULT_BLOCKSIZE("sserver.default.blocksize", "1M", PropertyType.BYTES,
       "Specifies a default blocksize for the scan server caches.", "2.1.0"),
   @Experimental
+  SSERV_GROUP_NAME("sserver.group", ScanServerSelector.DEFAULT_SCAN_SERVER_GROUP_NAME,
+      PropertyType.STRING,
+      "Optional group name that will be made available to the "
+          + "ScanServerSelector client plugin. Groups support at least two use cases:"
+          + " dedicating resources to scans and/or using different hardware for scans.",
+      "3.0.0"),
+  @Experimental
   SSERV_CACHED_TABLET_METADATA_EXPIRATION("sserver.cache.metadata.expiration", "5m",
       PropertyType.TIMEDURATION,
       "The time after which cached tablet metadata will be expired if not previously refreshed.",
@@ -534,6 +524,8 @@
   TSERV_TOTAL_MUTATION_QUEUE_MAX("tserver.total.mutation.queue.max", "5%", PropertyType.MEMORY,
       "The amount of memory used to store write-ahead-log mutations before flushing them.",
       "1.7.0"),
+  @ReplacedBy(property = SPLIT_MAXOPEN)
+  @Deprecated(since = "3.1")
   TSERV_TABLET_SPLIT_FINDMIDPOINT_MAXOPEN("tserver.tablet.split.midpoint.files.max", "300",
       PropertyType.COUNT,
       "To find a tablets split points, all RFiles are opened and their indexes"
@@ -547,29 +539,12 @@
           + "logs over this threshold is minor compacted.  Also any tablet referencing this many "
           + "logs or more will be compacted.",
       "2.1.0"),
-  @Deprecated(since = "2.1.0")
-  @ReplacedBy(property = Property.TSERV_WAL_MAX_REFERENCED)
-  TSERV_WALOG_MAX_REFERENCED("tserver.walog.max.referenced", "3", PropertyType.COUNT,
-      "When a tablet server has more than this many write ahead logs, any tablet referencing older "
-          + "logs over this threshold is minor compacted.  Also any tablet referencing this many "
-          + "logs or more will be compacted.",
-      "2.0.0"),
   TSERV_WAL_MAX_SIZE("tserver.wal.max.size", "1G", PropertyType.BYTES,
       "The maximum size for each write-ahead log. See comment for property"
           + " `tserver.memory.maps.max`.",
       "2.1.0"),
-  @Deprecated(since = "2.1.0")
-  @ReplacedBy(property = Property.TSERV_WAL_MAX_SIZE)
-  TSERV_WALOG_MAX_SIZE("tserver.walog.max.size", "1G", PropertyType.BYTES,
-      "The maximum size for each write-ahead log. See comment for property"
-          + " `tserver.memory.maps.max`.",
-      "1.3.5"),
   TSERV_WAL_MAX_AGE("tserver.wal.max.age", "24h", PropertyType.TIMEDURATION,
       "The maximum age for each write-ahead log.", "2.1.0"),
-  @Deprecated(since = "2.1.0")
-  @ReplacedBy(property = Property.TSERV_WAL_MAX_AGE)
-  TSERV_WALOG_MAX_AGE("tserver.walog.max.age", "24h", PropertyType.TIMEDURATION,
-      "The maximum age for each write-ahead log.", "1.6.6"),
   TSERV_WAL_TOLERATED_CREATION_FAILURES("tserver.wal.tolerated.creation.failures", "50",
       PropertyType.COUNT,
       "The maximum number of failures tolerated when creating a new write-ahead"
@@ -577,35 +552,14 @@
           + " number of failures consecutively trying to create a new write-ahead log"
           + " causes the TabletServer to exit.",
       "2.1.0"),
-  @Deprecated(since = "2.1.0")
-  @ReplacedBy(property = Property.TSERV_WAL_TOLERATED_CREATION_FAILURES)
-  TSERV_WALOG_TOLERATED_CREATION_FAILURES("tserver.walog.tolerated.creation.failures", "50",
-      PropertyType.COUNT,
-      "The maximum number of failures tolerated when creating a new write-ahead"
-          + " log. Negative values will allow unlimited creation failures. Exceeding this"
-          + " number of failures consecutively trying to create a new write-ahead log"
-          + " causes the TabletServer to exit.",
-      "1.7.1"),
   TSERV_WAL_TOLERATED_WAIT_INCREMENT("tserver.wal.tolerated.wait.increment", "1000ms",
       PropertyType.TIMEDURATION,
       "The amount of time to wait between failures to create or write a write-ahead log.", "2.1.0"),
-  @Deprecated(since = "2.1.0")
-  @ReplacedBy(property = Property.TSERV_WAL_TOLERATED_WAIT_INCREMENT)
-  TSERV_WALOG_TOLERATED_WAIT_INCREMENT("tserver.walog.tolerated.wait.increment", "1000ms",
-      PropertyType.TIMEDURATION,
-      "The amount of time to wait between failures to create or write a write-ahead log.", "1.7.1"),
   // Never wait longer than 5 mins for a retry
   TSERV_WAL_TOLERATED_MAXIMUM_WAIT_DURATION("tserver.wal.maximum.wait.duration", "5m",
       PropertyType.TIMEDURATION,
       "The maximum amount of time to wait after a failure to create or write a write-ahead log.",
       "2.1.0"),
-  // Never wait longer than 5 mins for a retry
-  @Deprecated(since = "2.1.0")
-  @ReplacedBy(property = Property.TSERV_WAL_TOLERATED_MAXIMUM_WAIT_DURATION)
-  TSERV_WALOG_TOLERATED_MAXIMUM_WAIT_DURATION("tserver.walog.maximum.wait.duration", "5m",
-      PropertyType.TIMEDURATION,
-      "The maximum amount of time to wait after a failure to create or write a write-ahead log.",
-      "1.7.1"),
   TSERV_SCAN_MAX_OPENFILES("tserver.scan.files.open.max", "100", PropertyType.COUNT,
       "Maximum total RFiles that all tablets in a tablet server can open for scans.", "1.4.0"),
   TSERV_MAX_IDLE("tserver.files.open.idle", "1m", PropertyType.TIMEDURATION,
@@ -663,19 +617,27 @@
       "The maximum number of concurrent tablet migrations for a tablet server.", "1.3.5"),
   TSERV_MAJC_DELAY("tserver.compaction.major.delay", "30s", PropertyType.TIMEDURATION,
       "Time a tablet server will sleep between checking which tablets need compaction.", "1.3.5"),
+  @Deprecated(since = "3.1")
+  @ReplacedBy(property = COMPACTION_SERVICE_PREFIX)
   TSERV_COMPACTION_SERVICE_PREFIX("tserver.compaction.major.service.", null, PropertyType.PREFIX,
       "Prefix for compaction services.", "2.1.0"),
+  @Deprecated(since = "3.1")
   TSERV_COMPACTION_SERVICE_ROOT_PLANNER("tserver.compaction.major.service.root.planner",
       DefaultCompactionPlanner.class.getName(), PropertyType.CLASSNAME,
       "Compaction planner for root tablet service.", "2.1.0"),
+  @Deprecated(since = "3.1")
   TSERV_COMPACTION_SERVICE_ROOT_RATE_LIMIT("tserver.compaction.major.service.root.rate.limit", "0B",
       PropertyType.BYTES,
       "Maximum number of bytes to read or write per second over all major"
-          + " compactions in this compaction service, or 0B for unlimited.",
+          + " compactions in this compaction service, or 0B for unlimited.  This property has"
+          + " been deprecated in anticipation of it being removed in a future release that"
+          + " removes the rate limiting feature.",
       "2.1.0"),
+  @Deprecated(since = "3.1")
   TSERV_COMPACTION_SERVICE_ROOT_MAX_OPEN(
       "tserver.compaction.major.service.root.planner.opts.maxOpen", "30", PropertyType.COUNT,
       "The maximum number of files a compaction will open.", "2.1.0"),
+  @Deprecated(since = "3.1")
   TSERV_COMPACTION_SERVICE_ROOT_EXECUTORS(
       "tserver.compaction.major.service.root.planner.opts.executors",
       "[{'name':'small','type':'internal','maxSize':'32M','numThreads':1},{'name':'huge','type':'internal','numThreads':1}]"
@@ -683,17 +645,23 @@
       PropertyType.STRING,
       "See {% jlink -f org.apache.accumulo.core.spi.compaction.DefaultCompactionPlanner %}.",
       "2.1.0"),
+  @Deprecated(since = "3.1")
   TSERV_COMPACTION_SERVICE_META_PLANNER("tserver.compaction.major.service.meta.planner",
       DefaultCompactionPlanner.class.getName(), PropertyType.CLASSNAME,
       "Compaction planner for metadata table.", "2.1.0"),
+  @Deprecated(since = "3.1")
   TSERV_COMPACTION_SERVICE_META_RATE_LIMIT("tserver.compaction.major.service.meta.rate.limit", "0B",
       PropertyType.BYTES,
       "Maximum number of bytes to read or write per second over all major"
-          + " compactions in this compaction service, or 0B for unlimited.",
+          + " compactions in this compaction service, or 0B for unlimited. This property has"
+          + " been deprecated in anticipation of it being removed in a future release that"
+          + " removes the rate limiting feature.",
       "2.1.0"),
+  @Deprecated(since = "3.1")
   TSERV_COMPACTION_SERVICE_META_MAX_OPEN(
       "tserver.compaction.major.service.meta.planner.opts.maxOpen", "30", PropertyType.COUNT,
       "The maximum number of files a compaction will open.", "2.1.0"),
+  @Deprecated(since = "3.1")
   TSERV_COMPACTION_SERVICE_META_EXECUTORS(
       "tserver.compaction.major.service.meta.planner.opts.executors",
       "[{'name':'small','type':'internal','maxSize':'32M','numThreads':2},{'name':'huge','type':'internal','numThreads':2}]"
@@ -701,20 +669,26 @@
       PropertyType.JSON,
       "See {% jlink -f org.apache.accumulo.core.spi.compaction.DefaultCompactionPlanner %}.",
       "2.1.0"),
+  @Deprecated(since = "3.1")
   TSERV_COMPACTION_SERVICE_DEFAULT_PLANNER(
       "tserver.compaction.major.service." + DEFAULT_COMPACTION_SERVICE_NAME + ".planner",
       DefaultCompactionPlanner.class.getName(), PropertyType.CLASSNAME,
       "Planner for default compaction service.", "2.1.0"),
+  @Deprecated(since = "3.1")
   TSERV_COMPACTION_SERVICE_DEFAULT_RATE_LIMIT(
       "tserver.compaction.major.service." + DEFAULT_COMPACTION_SERVICE_NAME + ".rate.limit", "0B",
       PropertyType.BYTES,
       "Maximum number of bytes to read or write per second over all major"
-          + " compactions in this compaction service, or 0B for unlimited.",
+          + " compactions in this compaction service, or 0B for unlimited. This property has"
+          + " been deprecated in anticipation of it being removed in a future release that"
+          + " removes the rate limiting feature.",
       "2.1.0"),
+  @Deprecated(since = "3.1")
   TSERV_COMPACTION_SERVICE_DEFAULT_MAX_OPEN(
       "tserver.compaction.major.service." + DEFAULT_COMPACTION_SERVICE_NAME
           + ".planner.opts.maxOpen",
       "10", PropertyType.COUNT, "The maximum number of files a compaction will open.", "2.1.0"),
+  @Deprecated(since = "3.1")
   TSERV_COMPACTION_SERVICE_DEFAULT_EXECUTORS(
       "tserver.compaction.major.service." + DEFAULT_COMPACTION_SERVICE_NAME
           + ".planner.opts.executors",
@@ -723,29 +697,10 @@
       PropertyType.STRING,
       "See {% jlink -f org.apache.accumulo.core.spi.compaction.DefaultCompactionPlanner %}.",
       "2.1.0"),
-  @Deprecated(since = "2.1.0", forRemoval = true)
-  @ReplacedBy(property = Property.TSERV_COMPACTION_SERVICE_DEFAULT_MAX_OPEN)
-  TSERV_MAJC_THREAD_MAXOPEN("tserver.compaction.major.thread.files.open.max", "10",
-      PropertyType.COUNT, "Max number of RFiles a major compaction thread can open at once.",
-      "1.4.0"),
-  @Deprecated(since = "2.1.0", forRemoval = true)
-  @ReplacedBy(property = Property.TSERV_COMPACTION_SERVICE_DEFAULT_EXECUTORS)
-  TSERV_MAJC_MAXCONCURRENT("tserver.compaction.major.concurrent.max", "3", PropertyType.COUNT,
-      "The maximum number of concurrent major compactions for a tablet server.", "1.3.5"),
-  @Deprecated(since = "2.1.0", forRemoval = true)
-  @ReplacedBy(property = Property.TSERV_COMPACTION_SERVICE_DEFAULT_RATE_LIMIT)
-  TSERV_MAJC_THROUGHPUT("tserver.compaction.major.throughput", "0B", PropertyType.BYTES,
-      "Maximum number of bytes to read or write per second over all major"
-          + " compactions within each compaction service, or 0B for unlimited.",
-      "1.8.0"),
   TSERV_MINC_MAXCONCURRENT("tserver.compaction.minor.concurrent.max", "4", PropertyType.COUNT,
       "The maximum number of concurrent minor compactions for a tablet server.", "1.3.5"),
-  @Deprecated(since = "2.1.0", forRemoval = true)
-  TSERV_MAJC_TRACE_PERCENT("tserver.compaction.major.trace.percent", "0.1", PropertyType.FRACTION,
-      "The percent of major compactions to trace.", "1.7.0"),
-  @Deprecated(since = "2.1.0", forRemoval = true)
-  TSERV_MINC_TRACE_PERCENT("tserver.compaction.minor.trace.percent", "0.1", PropertyType.FRACTION,
-      "The percent of minor compactions to trace.", "1.7.0"),
+  @Deprecated(since = "3.1")
+  @ReplacedBy(property = COMPACTION_WARN_TIME)
   TSERV_COMPACTION_WARN_TIME("tserver.compaction.warn.time", "10m", PropertyType.TIMEDURATION,
       "When a compaction has not made progress for this time period, a warning will be logged.",
       "1.6.0"),
@@ -753,12 +708,6 @@
       "The number of concurrent threads that will load bloom filters in the background. "
           + "Setting this to zero will make bloom filters load in the foreground.",
       "1.3.5"),
-  @Deprecated(since = "2.1.0", forRemoval = true)
-  TSERV_MONITOR_FS("tserver.monitor.fs", "false", PropertyType.BOOLEAN,
-      "When enabled the tserver will monitor file systems and kill itself when"
-          + " one switches from rw to ro. This is usually and indication that Linux has"
-          + " detected a bad disk.",
-      "1.3.5"),
   TSERV_MEMDUMP_DIR("tserver.dir.memdump", "/tmp", PropertyType.PATH,
       "A long running scan could possibly hold memory that has been minor"
           + " compacted. To prevent this, the in memory map is dumped to a local file"
@@ -766,23 +715,6 @@
           + " minor compacted file because it may have been modified by iterators. The"
           + " file dumped to the local dir is an exact copy of what was in memory.",
       "1.3.5"),
-  TSERV_BULK_PROCESS_THREADS("tserver.bulk.process.threads", "1", PropertyType.COUNT,
-      "The manager will task a tablet server with pre-processing a bulk import"
-          + " RFile prior to assigning it to the appropriate tablet servers. This"
-          + " configuration value controls the number of threads used to process the files.",
-      "1.4.0"),
-  TSERV_BULK_ASSIGNMENT_THREADS("tserver.bulk.assign.threads", "1", PropertyType.COUNT,
-      "The manager delegates bulk import RFile processing and assignment to"
-          + " tablet servers. After file has been processed, the tablet server will"
-          + " assign the file to the appropriate tablets on all servers. This property"
-          + " controls the number of threads used to communicate to the other servers.",
-      "1.4.0"),
-  TSERV_BULK_RETRY("tserver.bulk.retry.max", "5", PropertyType.COUNT,
-      "The number of times the tablet server will attempt to assign a RFile to"
-          + " a tablet as it migrates and splits.",
-      "1.4.0"),
-  TSERV_BULK_TIMEOUT("tserver.bulk.timeout", "5m", PropertyType.TIMEDURATION,
-      "The time to wait for a tablet server to process a bulk import request.", "1.4.3"),
   TSERV_HEALTH_CHECK_FREQ("tserver.health.check.interval", "30m", PropertyType.TIMEDURATION,
       "The time between tablet server health checks.", "2.1.0"),
   TSERV_MINTHREADS("tserver.server.threads.minimum", "20", PropertyType.COUNT,
@@ -818,16 +750,8 @@
       "1.5.0"),
   TSERV_WAL_SORT_MAX_CONCURRENT("tserver.wal.sort.concurrent.max", "2", PropertyType.COUNT,
       "The maximum number of threads to use to sort logs during recovery.", "2.1.0"),
-  @Deprecated(since = "2.1.0")
-  @ReplacedBy(property = Property.TSERV_WAL_SORT_MAX_CONCURRENT)
-  TSERV_RECOVERY_MAX_CONCURRENT("tserver.recovery.concurrent.max", "2", PropertyType.COUNT,
-      "The maximum number of threads to use to sort logs during recovery.", "1.5.0"),
   TSERV_WAL_SORT_BUFFER_SIZE("tserver.wal.sort.buffer.size", "10%", PropertyType.MEMORY,
       "The amount of memory to use when sorting logs during recovery.", "2.1.0"),
-  @Deprecated(since = "2.1.0")
-  @ReplacedBy(property = Property.TSERV_WAL_SORT_BUFFER_SIZE)
-  TSERV_SORT_BUFFER_SIZE("tserver.sort.buffer.size", "10%", PropertyType.MEMORY,
-      "The amount of memory to use when sorting logs during recovery.", "1.5.0"),
   TSERV_WAL_SORT_FILE_PREFIX("tserver.wal.sort.file.", null, PropertyType.PREFIX,
       "The rfile properties to use when sorting logs during recovery. Most of the properties"
           + " that begin with 'table.file' can be used here. For example, to set the compression"
@@ -836,7 +760,8 @@
   @Deprecated(since = "2.1.3")
   TSERV_WORKQ_THREADS("tserver.workq.threads", "2", PropertyType.COUNT,
       "The number of threads for the distributed work queue. These threads are"
-          + " used for copying failed bulk import RFiles. This property will be removed when bulk import V1 is removed.",
+          + " used for copying failed bulk import RFiles. Note that as of version 3.1.0 this property"
+          + " is not used and will be removed in a future release.",
       "1.4.2"),
   TSERV_WAL_SYNC("tserver.wal.sync", "true", PropertyType.BOOLEAN,
       "Use the SYNC_BLOCK create flag to sync WAL writes to disk. Prevents"
@@ -848,17 +773,6 @@
           + " warning along with the current stack trace. Meant to help debug stuck"
           + " assignments.",
       "1.6.2"),
-  @Deprecated(since = "2.1.0")
-  TSERV_REPLICATION_REPLAYERS("tserver.replication.replayer.", null, PropertyType.PREFIX,
-      "Allows configuration of implementation used to apply replicated data.", "1.7.0"),
-  @Deprecated(since = "2.1.0")
-  TSERV_REPLICATION_DEFAULT_HANDLER("tserver.replication.default.replayer",
-      "org.apache.accumulo.tserver.replication.BatchWriterReplicationReplayer",
-      PropertyType.CLASSNAME, "Default AccumuloReplicationReplayer implementation.", "1.7.0"),
-  @Deprecated(since = "2.1.0")
-  TSERV_REPLICATION_BW_REPLAYER_MEMORY("tserver.replication.batchwriter.replayer.memory", "50M",
-      PropertyType.BYTES, "Memory to provide to batchwriter to replay mutations for replication.",
-      "1.7.0"),
   TSERV_ASSIGNMENT_MAXCONCURRENT("tserver.assignment.concurrent.max", "2", PropertyType.COUNT,
       "The number of threads available to load tablets. Recoveries are still performed serially.",
       "1.7.0"),
@@ -886,6 +800,7 @@
       "The number of threads on each tablet server available to retrieve"
           + " summary data, that is not currently in cache, from RFiles.",
       "2.0.0"),
+  @Deprecated(since = "3.1")
   TSERV_LAST_LOCATION_MODE("tserver.last.location.mode", "compaction",
       PropertyType.LAST_LOCATION_MODE,
       "Describes how the system will record the 'last' location for tablets, which can be used for"
@@ -894,9 +809,10 @@
           + " 'assignment' is the mode, then the most recently assigned location will be recorded."
           + " The manager.startup.tserver properties might also need to be set to ensure the"
           + " tserver is available before tablets are initially assigned if the 'last' location is"
-          + " to be used.",
+          + " to be used. This property has been deprecated in anticipation of it being removed in"
+          + " a future release that removes major compactions from the TabletServer, rendering this"
+          + " feature moot.",
       "2.1.1"),
-
   // accumulo garbage collector properties
   GC_PREFIX("gc.", null, PropertyType.PREFIX,
       "Properties in this category affect the behavior of the accumulo garbage collector.",
@@ -914,17 +830,6 @@
       "The listening port for the garbage collector's monitor service.", "1.3.5"),
   GC_DELETE_THREADS("gc.threads.delete", "16", PropertyType.COUNT,
       "The number of threads used to delete RFiles and write-ahead logs.", "1.3.5"),
-  @Experimental
-  GC_REMOVE_IN_USE_CANDIDATES("gc.remove.in.use.candidates", "false", PropertyType.BOOLEAN,
-      "GC will remove deletion candidates that are in-use from the metadata location. "
-          + "This is expected to increase the speed of subsequent GC runs.",
-      "2.1.3"),
-  @Deprecated(since = "2.1.1", forRemoval = true)
-  GC_TRASH_IGNORE("gc.trash.ignore", "false", PropertyType.BOOLEAN,
-      "Do not use the Trash, even if it is configured.", "1.5.0"),
-  @Deprecated(since = "2.1.0", forRemoval = true)
-  GC_TRACE_PERCENT("gc.trace.percent", "0.01", PropertyType.FRACTION,
-      "Percent of gc cycles to trace.", "1.7.0"),
   GC_SAFEMODE("gc.safemode", "false", PropertyType.BOOLEAN,
       "Provides listing of files to be deleted but does not delete any files.", "2.1.0"),
   GC_USE_FULL_COMPACTION("gc.post.metadata.action", "flush", PropertyType.GC_POST_ACTION,
@@ -966,7 +871,7 @@
       "A comma-separated list of disallowed SSL Ciphers, see"
           + " monitor.ssl.include.ciphers to allow ciphers.",
       "1.6.1"),
-  MONITOR_SSL_INCLUDE_PROTOCOLS("monitor.ssl.include.protocols", "TLSv1.2", PropertyType.STRING,
+  MONITOR_SSL_INCLUDE_PROTOCOLS("monitor.ssl.include.protocols", "TLSv1.3", PropertyType.STRING,
       "A comma-separate list of allowed SSL protocols.", "1.5.3"),
   MONITOR_LOCK_CHECK_INTERVAL("monitor.lock.check.interval", "5s", PropertyType.TIMEDURATION,
       "The amount of time to sleep between checking for the Monitor ZooKeeper lock.", "1.5.1"),
@@ -979,41 +884,6 @@
           + " The resources that are used by default can be seen in"
           + " `accumulo/server/monitor/src/main/resources/templates/default.ftl`.",
       "2.0.0"),
-  @Deprecated(since = "2.1.0")
-  TRACE_PREFIX("trace.", null, PropertyType.PREFIX,
-      "Properties in this category affect the behavior of distributed tracing.", "1.3.5"),
-  @Deprecated(since = "2.1.0")
-  TRACE_SPAN_RECEIVERS("trace.span.receivers", "org.apache.accumulo.tracer.ZooTraceClient",
-      PropertyType.CLASSNAMELIST, "A list of span receiver classes to send trace spans.", "1.7.0"),
-  @Deprecated(since = "2.1.0")
-  TRACE_SPAN_RECEIVER_PREFIX("trace.span.receiver.", null, PropertyType.PREFIX,
-      "Prefix for span receiver configuration properties.", "1.7.0"),
-  @Deprecated(since = "2.1.0")
-  TRACE_ZK_PATH("trace.zookeeper.path", Constants.ZTRACERS, PropertyType.STRING,
-      "The zookeeper node where tracers are registered.", "1.7.0"),
-  @Deprecated(since = "2.1.0")
-  TRACE_PORT("trace.port.client", "12234", PropertyType.PORT,
-      "The listening port for the trace server.", "1.3.5"),
-  @Deprecated(since = "2.1.0")
-  TRACE_TABLE("trace.table", "trace", PropertyType.STRING,
-      "The name of the table to store distributed traces.", "1.3.5"),
-  @Deprecated(since = "2.1.0")
-  TRACE_USER("trace.user", "root", PropertyType.STRING,
-      "The name of the user to store distributed traces.", "1.3.5"),
-  @Sensitive
-  @Deprecated(since = "2.1.0")
-  TRACE_PASSWORD("trace.password", "secret", PropertyType.STRING,
-      "The password for the user used to store distributed traces.", "1.3.5"),
-  @Sensitive
-  @Deprecated(since = "2.1.0")
-  TRACE_TOKEN_PROPERTY_PREFIX("trace.token.property.", null, PropertyType.PREFIX,
-      "The prefix used to create a token for storing distributed traces. For"
-          + " each property required by trace.token.type, place this prefix in front of it.",
-      "1.5.0"),
-  @Deprecated(since = "2.1.0")
-  TRACE_TOKEN_TYPE("trace.token.type", PasswordToken.class.getName(), PropertyType.CLASSNAME,
-      "An AuthenticationToken type supported by the authorizer.", "1.5.0"),
-
   // per table properties
   TABLE_PREFIX("table.", null, PropertyType.PREFIX,
       "Properties in this category affect tablet server treatment of tablets,"
@@ -1042,23 +912,10 @@
       "Minimum ratio of total input size to maximum input RFile size for"
           + " running a major compaction.",
       "1.3.5"),
-  @Deprecated(since = "2.1.0", forRemoval = true)
-  TABLE_MAJC_COMPACTALL_IDLETIME("table.compaction.major.everything.idle", "1h",
-      PropertyType.TIMEDURATION,
-      "After a tablet has been idle (no mutations) for this time period it may"
-          + " have all of its RFiles compacted into one. There is no guarantee an idle"
-          + " tablet will be compacted. Compactions of idle tablets are only started"
-          + " when regular compactions are not running. Idle compactions only take"
-          + " place for tablets that have one or more RFiles.",
-      "1.3.5"),
   TABLE_SPLIT_THRESHOLD("table.split.threshold", "1G", PropertyType.BYTES,
       "A tablet is split when the combined size of RFiles exceeds this amount.", "1.3.5"),
   TABLE_MAX_END_ROW_SIZE("table.split.endrow.size.max", "10k", PropertyType.BYTES,
       "Maximum size of end row.", "1.7.0"),
-  @Deprecated(since = "2.0.0")
-  @ReplacedBy(property = Property.TSERV_WAL_MAX_REFERENCED)
-  TABLE_MINC_LOGS_MAX("table.compaction.minor.logs.threshold", "3", PropertyType.COUNT,
-      "This property is deprecated and replaced.", "1.3.5"),
   TABLE_MINC_COMPACT_IDLETIME("table.compaction.minor.idle", "5m", PropertyType.TIMEDURATION,
       "After a tablet has been idle (no mutations) for this time period it may have its "
           + "in-memory map flushed to disk in a minor compaction. There is no guarantee an idle "
@@ -1078,10 +935,13 @@
           + "specified time.  If a system compaction cancels a hold and runs, then the user compaction"
           + " can reselect and hold files after the system compaction runs.",
       "2.1.0"),
+  @Deprecated(since = "3.1")
   TABLE_COMPACTION_SELECTOR("table.compaction.selector", "", PropertyType.CLASSNAME,
       "A configurable selector for a table that can periodically select file for mandatory "
-          + "compaction, even if the files do not meet the compaction ratio.",
+          + "compaction, even if the files do not meet the compaction ratio. This option was deprecated in "
+          + "3.1, see the CompactionKind.SELECTOR enum javadoc for details.",
       "2.1.0"),
+  @Deprecated(since = "3.1")
   TABLE_COMPACTION_SELECTOR_OPTS("table.compaction.selector.opts.", null, PropertyType.PREFIX,
       "Options for the table compaction dispatcher.", "2.1.0"),
   TABLE_COMPACTION_CONFIGURER("table.compaction.configurer", "", PropertyType.CLASSNAME,
@@ -1089,15 +949,6 @@
       "2.1.0"),
   TABLE_COMPACTION_CONFIGURER_OPTS("table.compaction.configurer.opts.", null, PropertyType.PREFIX,
       "Options for the table compaction configuror.", "2.1.0"),
-  @Deprecated(since = "2.1.0", forRemoval = true)
-  @ReplacedBy(property = TABLE_COMPACTION_SELECTOR)
-  TABLE_COMPACTION_STRATEGY("table.majc.compaction.strategy", "", PropertyType.CLASSNAME,
-      "See {% jlink -f org.apache.accumulo.core.spi.compaction %}.", "1.6.0"),
-  @Deprecated(since = "2.1.0", forRemoval = true)
-  @ReplacedBy(property = TABLE_COMPACTION_SELECTOR_OPTS)
-  TABLE_COMPACTION_STRATEGY_PREFIX("table.majc.compaction.strategy.opts.", null,
-      PropertyType.PREFIX,
-      "Properties in this category are used to configure the compaction strategy.", "1.6.0"),
   // Crypto-related properties
   @Experimental
   TABLE_CRYPTO_PREFIX("table.crypto.opts.", null, PropertyType.PREFIX,
@@ -1266,31 +1117,10 @@
       "1.3.5"),
   TABLE_FORMATTER_CLASS("table.formatter", DefaultFormatter.class.getName(), PropertyType.STRING,
       "The Formatter class to apply on results in the shell.", "1.4.0"),
-  @Deprecated(since = "2.1.0")
-  TABLE_INTERPRETER_CLASS("table.interepreter",
-      org.apache.accumulo.core.util.interpret.DefaultScanInterpreter.class.getName(),
-      PropertyType.STRING,
-      "The ScanInterpreter class to apply on scan arguments in the shell. "
-          + "Note that this property is deprecated and will be removed in a future version.",
-      "1.5.0"),
   TABLE_CLASSLOADER_CONTEXT("table.class.loader.context", "", PropertyType.STRING,
       "The context to use for loading per-table resources, such as iterators"
           + " from the configured factory in `general.context.class.loader.factory`.",
       "2.1.0"),
-  @Deprecated(since = "2.1.0", forRemoval = true)
-  @ReplacedBy(property = TABLE_CLASSLOADER_CONTEXT)
-  TABLE_CLASSPATH("table.classpath.context", "", PropertyType.STRING,
-      "Per table classpath context.", "1.5.0"),
-  @Deprecated(since = "2.1.0")
-  TABLE_REPLICATION("table.replication", "false", PropertyType.BOOLEAN,
-      "Is replication enabled for the given table.", "1.7.0"),
-  @Deprecated(since = "2.1.0")
-  TABLE_REPLICATION_TARGET("table.replication.target.", null, PropertyType.PREFIX,
-      "Enumerate a mapping of other systems which this table should replicate"
-          + " their data to. The key suffix is the identifying cluster name and the"
-          + " value is an identifier for a location on the target system, e.g. the ID"
-          + " of the table on the target to replicate to.",
-      "1.7.0"),
   TABLE_SAMPLER("table.sampler", "", PropertyType.CLASSNAME,
       "The name of a class that implements org.apache.accumulo.core.Sampler."
           + " Setting this option enables storing a sample of data which can be"
@@ -1331,122 +1161,6 @@
           + "constraint.",
       "2.0.0"),
 
-  // VFS ClassLoader properties
-
-  // this property shouldn't be used directly; it exists solely to document the default value
-  // defined by its use in AccumuloVFSClassLoader when generating the property documentation
-  @Deprecated(since = "2.1.0", forRemoval = true)
-  VFS_CLASSLOADER_SYSTEM_CLASSPATH_PROPERTY(
-      org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader.VFS_CLASSLOADER_SYSTEM_CLASSPATH_PROPERTY,
-      "", PropertyType.STRING,
-      "Configuration for a system level vfs classloader. Accumulo jar can be"
-          + " configured here and loaded out of HDFS.",
-      "1.5.0"),
-  @Deprecated(since = "2.1.0", forRemoval = true)
-  VFS_CONTEXT_CLASSPATH_PROPERTY(
-      org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader.VFS_CONTEXT_CLASSPATH_PROPERTY,
-      null, PropertyType.PREFIX,
-      "Properties in this category are define a classpath. These properties"
-          + " start  with the category prefix, followed by a context name. The value is"
-          + " a comma separated list of URIs. Supports full regex on filename alone."
-          + " For example, general.vfs.context.classpath.cx1=hdfs://nn1:9902/mylibdir/*.jar."
-          + " You can enable post delegation for a context, which will load classes from the"
-          + " context first instead of the parent first. Do this by setting"
-          + " `general.vfs.context.classpath.<name>.delegation=post`, where `<name>` is"
-          + " your context name. If delegation is not specified, it defaults to loading"
-          + " from parent classloader first.",
-      "1.5.0"),
-
-  // this property shouldn't be used directly; it exists solely to document the default value
-  // defined by its use in AccumuloVFSClassLoader when generating the property documentation
-  @Deprecated(since = "2.1.0", forRemoval = true)
-  VFS_CLASSLOADER_CACHE_DIR(
-      org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader.VFS_CACHE_DIR,
-      "${java.io.tmpdir}", PropertyType.ABSOLUTEPATH,
-      "The base directory to use for the vfs cache. The actual cached files will be located"
-          + " in a subdirectory, `accumulo-vfs-cache-<jvmProcessName>-${user.name}`, where"
-          + " `<jvmProcessName>` is determined by the JVM's internal management engine."
-          + " The cache will keep a soft reference to all of the classes loaded in the VM."
-          + " This should be on local disk on each node with sufficient space.",
-      "1.5.0"),
-
-  // General properties for configuring replication
-  @Deprecated(since = "2.1.0")
-  REPLICATION_PREFIX("replication.", null, PropertyType.PREFIX,
-      "Properties in this category affect the replication of data to other Accumulo instances.",
-      "1.7.0"),
-  @Deprecated(since = "2.1.0")
-  REPLICATION_PEERS("replication.peer.", null, PropertyType.PREFIX,
-      "Properties in this category control what systems data can be replicated to.", "1.7.0"),
-  @Deprecated(since = "2.1.0")
-  REPLICATION_PEER_USER("replication.peer.user.", null, PropertyType.PREFIX,
-      "The username to provide when authenticating with the given peer.", "1.7.0"),
-  @Sensitive
-  @Deprecated(since = "2.1.0")
-  REPLICATION_PEER_PASSWORD("replication.peer.password.", null, PropertyType.PREFIX,
-      "The password to provide when authenticating with the given peer.", "1.7.0"),
-  @Deprecated(since = "2.1.0")
-  REPLICATION_PEER_KEYTAB("replication.peer.keytab.", null, PropertyType.PREFIX,
-      "The keytab to use when authenticating with the given peer.", "1.7.0"),
-  @Deprecated(since = "2.1.0")
-  REPLICATION_NAME("replication.name", "", PropertyType.STRING,
-      "Name of this cluster with respect to replication. Used to identify this"
-          + " instance from other peers.",
-      "1.7.0"),
-  @Deprecated(since = "2.1.0")
-  REPLICATION_MAX_WORK_QUEUE("replication.max.work.queue", "1000", PropertyType.COUNT,
-      "Upper bound of the number of files queued for replication.", "1.7.0"),
-  @Deprecated(since = "2.1.0")
-  REPLICATION_WORK_ASSIGNMENT_SLEEP("replication.work.assignment.sleep", "30s",
-      PropertyType.TIMEDURATION, "Amount of time to sleep between replication work assignment.",
-      "1.7.0"),
-  @Deprecated(since = "2.1.0")
-  REPLICATION_WORKER_THREADS("replication.worker.threads", "4", PropertyType.COUNT,
-      "Size of the threadpool that each tabletserver devotes to replicating data.", "1.7.0"),
-  @Deprecated(since = "2.1.0")
-  REPLICATION_RECEIPT_SERVICE_PORT("replication.receipt.service.port", "10002", PropertyType.PORT,
-      "Listen port used by thrift service in tserver listening for replication.", "1.7.0"),
-  @Deprecated(since = "2.1.0")
-  REPLICATION_WORK_ATTEMPTS("replication.work.attempts", "10", PropertyType.COUNT,
-      "Number of attempts to try to replicate some data before giving up and"
-          + " letting it naturally be retried later.",
-      "1.7.0"),
-  @Deprecated(since = "2.1.0")
-  REPLICATION_MIN_THREADS("replication.receiver.min.threads", "1", PropertyType.COUNT,
-      "Minimum number of threads for replication.", "1.7.0"),
-  @Deprecated(since = "2.1.0")
-  REPLICATION_THREADCHECK("replication.receiver.threadcheck.time", "30s", PropertyType.TIMEDURATION,
-      "The time between adjustments of the replication thread pool.", "1.7.0"),
-  @Deprecated(since = "2.1.0")
-  REPLICATION_MAX_UNIT_SIZE("replication.max.unit.size", "64M", PropertyType.BYTES,
-      "Maximum size of data to send in a replication message.", "1.7.0"),
-  @Deprecated(since = "2.1.0")
-  REPLICATION_WORK_ASSIGNER("replication.work.assigner",
-      "org.apache.accumulo.manager.replication.UnorderedWorkAssigner", PropertyType.CLASSNAME,
-      "Replication WorkAssigner implementation to use.", "1.7.0"),
-  @Deprecated(since = "2.1.0")
-  REPLICATION_DRIVER_DELAY("replication.driver.delay", "0s", PropertyType.TIMEDURATION,
-      "Amount of time to wait before the replication work loop begins in the manager.", "1.7.0"),
-  @Deprecated(since = "2.1.0")
-  REPLICATION_WORK_PROCESSOR_DELAY("replication.work.processor.delay", "0s",
-      PropertyType.TIMEDURATION,
-      "Amount of time to wait before first checking for replication work, not"
-          + " useful outside of tests.",
-      "1.7.0"),
-  @Deprecated(since = "2.1.0")
-  REPLICATION_WORK_PROCESSOR_PERIOD("replication.work.processor.period", "0s",
-      PropertyType.TIMEDURATION,
-      "Amount of time to wait before re-checking for replication work, not"
-          + " useful outside of tests.",
-      "1.7.0"),
-  @Deprecated(since = "2.1.0", forRemoval = true)
-  REPLICATION_TRACE_PERCENT("replication.trace.percent", "0.1", PropertyType.FRACTION,
-      "The sampling percentage to use for replication traces.", "1.7.0"),
-  @Deprecated(since = "2.1.0")
-  REPLICATION_RPC_TIMEOUT("replication.rpc.timeout", "2m", PropertyType.TIMEDURATION,
-      "Amount of time for a single replication RPC call to last before failing"
-          + " the attempt. See replication.work.attempts.",
-      "1.7.4"),
   // Compactor properties
   @Experimental
   COMPACTOR_PREFIX("compactor.", null, PropertyType.PREFIX,
@@ -1481,6 +1195,9 @@
   @Experimental
   COMPACTOR_MAX_MESSAGE_SIZE("compactor.message.size.max", "10M", PropertyType.BYTES,
       "The maximum size of a message that can be sent to a tablet server.", "2.1.0"),
+  @Experimental
+  COMPACTOR_QUEUE_NAME("compactor.queue", "", PropertyType.STRING,
+      "The queue for which this Compactor will perform compactions.", "3.0.0"),
   // CompactionCoordinator properties
   @Experimental
   COMPACTION_COORDINATOR_PREFIX("compaction.coordinator.", null, PropertyType.PREFIX,
@@ -1529,54 +1246,7 @@
   @Experimental
   COMPACTION_COORDINATOR_TSERVER_COMPACTION_CHECK_INTERVAL(
       "compaction.coordinator.tserver.check.interval", "1m", PropertyType.TIMEDURATION,
-      "The interval at which to check the tservers for external compactions.", "2.1.0"),
-  // deprecated properties grouped at the end to reference property that replaces them
-  @Deprecated(since = "1.6.0")
-  @ReplacedBy(property = INSTANCE_VOLUMES)
-  INSTANCE_DFS_URI("instance.dfs.uri", "", PropertyType.URI,
-      "A url accumulo should use to connect to DFS. If this is empty, accumulo"
-          + " will obtain this information from the hadoop configuration. This property"
-          + " will only be used when creating new files if instance.volumes is empty."
-          + " After an upgrade to 1.6.0 Accumulo will start using absolute paths to"
-          + " reference files. Files created before a 1.6.0 upgrade are referenced via"
-          + " relative paths. Relative paths will always be resolved using this config"
-          + " (if empty using the hadoop config).",
-      "1.4.0"),
-  @Deprecated(since = "1.6.0")
-  @ReplacedBy(property = INSTANCE_VOLUMES)
-  INSTANCE_DFS_DIR("instance.dfs.dir", "/accumulo", PropertyType.ABSOLUTEPATH,
-      "HDFS directory in which accumulo instance will run. "
-          + "Do not change after accumulo is initialized.",
-      "1.3.5"),
-  @Deprecated(since = "2.0.0")
-  GENERAL_CLASSPATHS(org.apache.accumulo.start.classloader.AccumuloClassLoader.GENERAL_CLASSPATHS,
-      "", PropertyType.STRING,
-      "The class path should instead be configured"
-          + " by the launch environment (for example, accumulo-env.sh). A list of all"
-          + " of the places to look for a class. Order does matter, as it will look for"
-          + " the jar starting in the first location to the last. Supports full regex"
-          + " on filename alone.",
-      "1.3.5"),
-  @Deprecated(since = "1.7.0")
-  @ReplacedBy(property = TABLE_DURABILITY)
-  TSERV_WAL_SYNC_METHOD("tserver.wal.sync.method", "hsync", PropertyType.STRING,
-      "Use table.durability instead.", "1.5.2"),
-  @Deprecated(since = "1.7.0")
-  @ReplacedBy(property = TABLE_DURABILITY)
-  TABLE_WALOG_ENABLED("table.walog.enabled", "true", PropertyType.BOOLEAN,
-      "Use table.durability=none instead.", "1.3.5"),
-  @Deprecated(since = "2.0.0")
-  @ReplacedBy(property = TSERV_SCAN_EXECUTORS_DEFAULT_THREADS)
-  TSERV_READ_AHEAD_MAXCONCURRENT("tserver.readahead.concurrent.max", "16", PropertyType.COUNT,
-      "The maximum number of concurrent read ahead that will execute. This "
-          + "effectively limits the number of long running scans that can run concurrently "
-          + "per tserver.",
-      "1.3.5"),
-  @Deprecated(since = "2.0.0")
-  @ReplacedBy(property = TSERV_SCAN_EXECUTORS_META_THREADS)
-  TSERV_METADATA_READ_AHEAD_MAXCONCURRENT("tserver.metadata.readahead.concurrent.max", "8",
-      PropertyType.COUNT, "The maximum number of concurrent metadata read ahead that will execute.",
-      "1.3.5");
+      "The interval at which to check the tservers for external compactions.", "2.1.0");
 
   private final String key;
   private final String defaultValue;
@@ -1733,6 +1403,8 @@
     ReplacedBy rb = getAnnotation(ReplacedBy.class);
     if (rb != null) {
       replacedBy = rb.property();
+    } else {
+      isReplaced = false;
     }
     annotationsComputed = true;
   }
@@ -1831,8 +1503,6 @@
         && (key.startsWith(Property.TABLE_CONSTRAINT_PREFIX.getKey())
             || key.startsWith(Property.TABLE_ITERATOR_PREFIX.getKey())
             || key.startsWith(Property.TABLE_LOCALITY_GROUP_PREFIX.getKey())
-            || key.startsWith(Property.TABLE_COMPACTION_STRATEGY_PREFIX.getKey())
-            || key.startsWith(Property.TABLE_REPLICATION_TARGET.getKey())
             || key.startsWith(Property.TABLE_ARBITRARY_PROP_PREFIX.getKey())
             || key.startsWith(TABLE_SAMPLER_OPTS.getKey())
             || key.startsWith(TABLE_SUMMARIZER_PREFIX.getKey())
@@ -1892,15 +1562,14 @@
     // white list prefixes
     return key.startsWith(Property.TABLE_PREFIX.getKey())
         || key.startsWith(Property.TSERV_PREFIX.getKey())
+        || key.startsWith(Property.COMPACTION_SERVICE_PREFIX.getKey())
         || key.startsWith(Property.SSERV_PREFIX.getKey())
         || key.startsWith(Property.MANAGER_PREFIX.getKey())
-        || key.startsWith(Property.MASTER_PREFIX.getKey())
         || key.startsWith(Property.GC_PREFIX.getKey())
         || key.startsWith(Property.GENERAL_ARBITRARY_PROP_PREFIX.getKey())
+        || key.equals(Property.COMPACTION_WARN_TIME.getKey())
         || key.equals(Property.GENERAL_FILE_NAME_ALLOCATION_BATCH_SIZE_MIN.getKey())
-        || key.equals(Property.GENERAL_FILE_NAME_ALLOCATION_BATCH_SIZE_MAX.getKey())
-        || key.startsWith(VFS_CONTEXT_CLASSPATH_PROPERTY.getKey())
-        || key.startsWith(REPLICATION_PREFIX.getKey());
+        || key.equals(Property.GENERAL_FILE_NAME_ALLOCATION_BATCH_SIZE_MAX.getKey());
   }
 
   /**
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/PropertyType.java b/core/src/main/java/org/apache/accumulo/core/conf/PropertyType.java
index a0c7f55..0181db3 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/PropertyType.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/PropertyType.java
@@ -112,11 +112,8 @@
           + "Substitutions of the ACCUMULO_HOME environment variable can be done in the system "
           + "config file using '${env:ACCUMULO_HOME}' or similar."),
 
-  // VFS_CLASSLOADER_CACHE_DIR's default value is a special case, for documentation purposes
-  @SuppressWarnings("removal")
   ABSOLUTEPATH("absolute path",
-      x -> x == null || x.trim().isEmpty() || new Path(x.trim()).isAbsolute()
-          || x.equals(Property.VFS_CLASSLOADER_CACHE_DIR.getDefaultValue()),
+      x -> x == null || x.trim().isEmpty() || new Path(x.trim()).isAbsolute(),
       "An absolute filesystem path. The filesystem depends on the property."
           + " This is the same as path, but enforces that its root is explicitly specified."),
 
@@ -248,7 +245,7 @@
         || (suffixCheck.test(x) && new Bounds(lowerBound, upperBound).test(stripUnits.apply(x)));
   }
 
-  private static final Pattern SUFFIX_REGEX = Pattern.compile("[^\\d]*$");
+  private static final Pattern SUFFIX_REGEX = Pattern.compile("\\D*$"); // match non-digits at end
   private static final Function<String,String> stripUnits =
       x -> x == null ? null : SUFFIX_REGEX.matcher(x.trim()).replaceAll("");
 
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/SiteConfiguration.java b/core/src/main/java/org/apache/accumulo/core/conf/SiteConfiguration.java
index 196486f..ed1775c 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/SiteConfiguration.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/SiteConfiguration.java
@@ -88,14 +88,6 @@
     }
 
     public OverridesOption fromEnv() {
-      URL siteUrl = SiteConfiguration.class.getClassLoader().getResource("accumulo-site.xml");
-      if (siteUrl != null) {
-        throw new IllegalArgumentException("Found deprecated config file 'accumulo-site.xml' on "
-            + "classpath. Since 2.0.0, this file was replaced by 'accumulo.properties'. Run the "
-            + "following command to convert an old 'accumulo-site.xml' file to the new format: "
-            + "accumulo convert-config -x /old/accumulo-site.xml -p /new/accumulo.properties");
-      }
-
       String configFile = System.getProperty("accumulo.properties", "accumulo.properties");
       if (configFile.startsWith("file://")) {
         File f;
@@ -166,9 +158,6 @@
       config.addConfiguration(overrideConfig);
       config.addConfiguration(propsFileConfig);
 
-      // Make sure any deprecated property names aren't using both the old and new name.
-      DeprecatedPropertyUtil.sanityCheckManagerProperties(config);
-
       var result = new HashMap<String,String>();
       config.getKeys().forEachRemaining(orig -> {
         String resolved = DeprecatedPropertyUtil.getReplacementName(orig, (log, replacement) -> {
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/cluster/ClusterConfigParser.java b/core/src/main/java/org/apache/accumulo/core/conf/cluster/ClusterConfigParser.java
index 5790fa7..e005701 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/cluster/ClusterConfigParser.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/cluster/ClusterConfigParser.java
@@ -72,14 +72,12 @@
         (parentKey == null || parentKey.equals("")) ? "" : parentKey + addTheDot(parentKey);
     if (value instanceof String) {
       results.put(parent + key, (String) value);
-      return;
     } else if (value instanceof List) {
       ((List<?>) value).forEach(l -> {
         if (l instanceof String) {
           // remove the [] at the ends of toString()
           String val = value.toString();
           results.put(parent + key, val.substring(1, val.length() - 1).replace(", ", " "));
-          return;
         } else {
           flatten(parent, key, l, results);
         }
@@ -90,9 +88,8 @@
       map.forEach((k, v) -> flatten(parent + key, k, v, results));
     } else if (value instanceof Number) {
       results.put(parent + key, value.toString());
-      return;
     } else {
-      throw new RuntimeException("Unhandled object type: " + value.getClass());
+      throw new IllegalStateException("Unhandled object type: " + value.getClass());
     }
   }
 
@@ -109,7 +106,7 @@
         out.printf(PROPERTY_FORMAT, section.toUpperCase() + "_HOSTS", config.get(section));
       } else {
         if (section.equals("manager") || section.equals("tserver")) {
-          throw new RuntimeException("Required configuration section is missing: " + section);
+          throw new IllegalStateException("Required configuration section is missing: " + section);
         }
         System.err.println("WARN: " + section + " is missing");
       }
diff --git a/core/src/main/java/org/apache/accumulo/core/constraints/Constraint.java b/core/src/main/java/org/apache/accumulo/core/constraints/Constraint.java
deleted file mode 100644
index 1106452..0000000
--- a/core/src/main/java/org/apache/accumulo/core/constraints/Constraint.java
+++ /dev/null
@@ -1,98 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.constraints;
-
-import java.util.List;
-
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.dataImpl.KeyExtent;
-import org.apache.accumulo.core.security.AuthorizationContainer;
-
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
-/**
- * This class is replaced by {@link org.apache.accumulo.core.data.constraints.Constraint}
- *
- * @deprecated since 2.1.0 Use {@link org.apache.accumulo.core.data.constraints.Constraint}
- */
-@Deprecated(since = "2.1.0")
-@SuppressFBWarnings(value = "NM_SAME_SIMPLE_NAME_AS_INTERFACE",
-    justification = "Same name used for compatibility during deprecation cycle")
-public interface Constraint extends org.apache.accumulo.core.data.constraints.Constraint {
-
-  /**
-   * The environment within which a constraint exists.
-   */
-  interface Environment extends org.apache.accumulo.core.data.constraints.Constraint.Environment {
-    /**
-     * Gets the key extent of the environment.
-     *
-     * @return key extent
-     */
-    KeyExtent getExtent();
-
-    /**
-     * Gets the user within the environment.
-     *
-     * @return user
-     */
-    @Override
-    String getUser();
-
-    /**
-     * Gets the authorizations in the environment.
-     *
-     * @return authorizations
-     */
-    @Override
-    AuthorizationContainer getAuthorizationsContainer();
-  }
-
-  /**
-   * Gets a short, one-sentence description of what a given violation code means.
-   *
-   * @param violationCode numeric violation code
-   * @return matching violation description
-   */
-  @Override
-  String getViolationDescription(short violationCode);
-
-  /**
-   * Checks a mutation for constraint violations. If the mutation contains no violations, returns
-   * null. Otherwise, returns a list of violation codes.
-   *
-   * Violation codes must be non-negative. Negative violation codes are reserved for system use.
-   *
-   * @param env constraint environment
-   * @param mutation mutation to check
-   * @return list of violation codes, or null if none
-   */
-  List<Short> check(Environment env, Mutation mutation);
-
-  /**
-   * Implemented for backwards compatibility.
-   *
-   * @since 2.1.0
-   */
-  @Override
-  default List<Short> check(org.apache.accumulo.core.data.constraints.Constraint.Environment env,
-      Mutation mutation) {
-    return check((Environment) env, mutation);
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/constraints/DefaultKeySizeConstraint.java b/core/src/main/java/org/apache/accumulo/core/constraints/DefaultKeySizeConstraint.java
deleted file mode 100644
index 0c09b46..0000000
--- a/core/src/main/java/org/apache/accumulo/core/constraints/DefaultKeySizeConstraint.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.constraints;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.accumulo.core.data.ColumnUpdate;
-import org.apache.accumulo.core.data.Mutation;
-
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
-/**
- * A constraints that limits the size of keys to 1mb.
- *
- * @deprecated since 2.1.0 Use
- *             {@link org.apache.accumulo.core.data.constraints.DefaultKeySizeConstraint}
- */
-@Deprecated(since = "2.1.0")
-@SuppressFBWarnings(value = "NM_SAME_SIMPLE_NAME_AS_SUPERCLASS",
-    justification = "Same name used for compatibility during deprecation cycle")
-public class DefaultKeySizeConstraint extends
-    org.apache.accumulo.core.data.constraints.DefaultKeySizeConstraint implements Constraint {
-
-  protected static final short MAX__KEY_SIZE_EXCEEDED_VIOLATION = 1;
-  protected static final long maxSize = 1048576; // 1MB default size
-
-  @Override
-  public String getViolationDescription(short violationCode) {
-
-    switch (violationCode) {
-      case MAX__KEY_SIZE_EXCEEDED_VIOLATION:
-        return "Key was larger than 1MB";
-    }
-
-    return null;
-  }
-
-  static final List<Short> NO_VIOLATIONS = new ArrayList<>();
-
-  @Override
-  public List<Short> check(Constraint.Environment env, Mutation mutation) {
-
-    // fast size check
-    if (mutation.numBytes() < maxSize) {
-      return NO_VIOLATIONS;
-    }
-
-    List<Short> violations = new ArrayList<>();
-
-    for (ColumnUpdate cu : mutation.getUpdates()) {
-      int size = mutation.getRow().length;
-      size += cu.getColumnFamily().length;
-      size += cu.getColumnQualifier().length;
-      size += cu.getColumnVisibility().length;
-
-      if (size > maxSize) {
-        violations.add(MAX__KEY_SIZE_EXCEEDED_VIOLATION);
-      }
-    }
-
-    return violations;
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/constraints/NoDeleteConstraint.java b/core/src/main/java/org/apache/accumulo/core/constraints/NoDeleteConstraint.java
deleted file mode 100644
index f0f5db5..0000000
--- a/core/src/main/java/org/apache/accumulo/core/constraints/NoDeleteConstraint.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.constraints;
-
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.accumulo.core.data.ColumnUpdate;
-import org.apache.accumulo.core.data.Mutation;
-
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
-/**
- * This constraint ensures mutations do not have deletes.
- *
- * @since 2.0.0
- * @deprecated since 2.1.0 Use {@link org.apache.accumulo.core.data.constraints.NoDeleteConstraint}
- */
-@Deprecated(since = "2.1.0")
-@SuppressFBWarnings(value = "NM_SAME_SIMPLE_NAME_AS_SUPERCLASS",
-    justification = "Same name used for compatibility during deprecation cycle")
-public class NoDeleteConstraint extends org.apache.accumulo.core.data.constraints.NoDeleteConstraint
-    implements Constraint {
-
-  @Override
-  public String getViolationDescription(short violationCode) {
-    if (violationCode == 1) {
-      return "Deletes are not allowed";
-    }
-    return null;
-  }
-
-  @Override
-  public List<Short> check(Constraint.Environment env, Mutation mutation) {
-    List<ColumnUpdate> updates = mutation.getUpdates();
-    for (ColumnUpdate update : updates) {
-      if (update.isDeleted()) {
-        return Collections.singletonList((short) 1);
-      }
-    }
-    return null;
-  }
-
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/constraints/VisibilityConstraint.java b/core/src/main/java/org/apache/accumulo/core/constraints/VisibilityConstraint.java
deleted file mode 100644
index 2233dff..0000000
--- a/core/src/main/java/org/apache/accumulo/core/constraints/VisibilityConstraint.java
+++ /dev/null
@@ -1,107 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.constraints;
-
-import static java.nio.charset.StandardCharsets.UTF_8;
-
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-
-import org.apache.accumulo.core.data.ColumnUpdate;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.security.ColumnVisibility;
-import org.apache.accumulo.core.security.VisibilityEvaluator;
-import org.apache.accumulo.core.security.VisibilityParseException;
-import org.apache.accumulo.core.util.BadArgumentException;
-
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
-/**
- * A constraint that checks the visibility of columns against the actor's authorizations. Violation
- * codes:
- * <ul>
- * <li>1 = failure to parse visibility expression</li>
- * <li>2 = insufficient authorization</li>
- * </ul>
- *
- * @deprecated since 2.1.0 Use
- *             {@link org.apache.accumulo.core.data.constraints.VisibilityConstraint}
- */
-@Deprecated(since = "2.1.0")
-@SuppressFBWarnings(value = "NM_SAME_SIMPLE_NAME_AS_SUPERCLASS",
-    justification = "Same name used for compatibility during deprecation cycle")
-public class VisibilityConstraint
-    extends org.apache.accumulo.core.data.constraints.VisibilityConstraint implements Constraint {
-
-  @Override
-  public String getViolationDescription(short violationCode) {
-    switch (violationCode) {
-      case 1:
-        return "Malformed column visibility";
-      case 2:
-        return "User does not have authorization on column visibility";
-    }
-
-    return null;
-  }
-
-  @Override
-  public List<Short> check(Constraint.Environment env, Mutation mutation) {
-    List<ColumnUpdate> updates = mutation.getUpdates();
-
-    HashSet<String> ok = null;
-    if (updates.size() > 1) {
-      ok = new HashSet<>();
-    }
-
-    VisibilityEvaluator ve = null;
-
-    for (ColumnUpdate update : updates) {
-
-      byte[] cv = update.getColumnVisibility();
-      if (cv.length > 0) {
-        String key = null;
-        if (ok != null && ok.contains(key = new String(cv, UTF_8))) {
-          continue;
-        }
-
-        try {
-
-          if (ve == null) {
-            ve = new VisibilityEvaluator(env.getAuthorizationsContainer());
-          }
-
-          if (!ve.evaluate(new ColumnVisibility(cv))) {
-            return Collections.singletonList((short) 2);
-          }
-
-        } catch (BadArgumentException | VisibilityParseException bae) {
-          return Collections.singletonList((short) 1);
-        }
-
-        if (ok != null) {
-          ok.add(key);
-        }
-      }
-    }
-
-    return null;
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/crypto/streams/BlockedInputStream.java b/core/src/main/java/org/apache/accumulo/core/crypto/streams/BlockedInputStream.java
index 3e37f6f..f3dccea 100644
--- a/core/src/main/java/org/apache/accumulo/core/crypto/streams/BlockedInputStream.java
+++ b/core/src/main/java/org/apache/accumulo/core/crypto/streams/BlockedInputStream.java
@@ -38,7 +38,7 @@
 
   public BlockedInputStream(InputStream in, int blockSize, int maxSize) {
     if (blockSize == 0) {
-      throw new RuntimeException("Invalid block size");
+      throw new IllegalArgumentException("Invalid block size");
     }
     if (in instanceof DataInputStream) {
       this.in = (DataInputStream) in;
@@ -67,7 +67,7 @@
     if (readPos == array.length) {
       readPos = 0;
     } else if (readPos > array.length) {
-      throw new RuntimeException(
+      throw new IllegalStateException(
           "Unexpected state, this should only ever increase or cycle on the boundary!");
     }
     return toRet;
@@ -121,7 +121,7 @@
       finished = true;
       return false;
     } else if (size == 0) {
-      throw new RuntimeException(
+      throw new IllegalStateException(
           "Empty block written, this shouldn't happen with this BlockedOutputStream.");
     }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/crypto/streams/BlockedOutputStream.java b/core/src/main/java/org/apache/accumulo/core/crypto/streams/BlockedOutputStream.java
index 05de9f7..6fd59b7 100644
--- a/core/src/main/java/org/apache/accumulo/core/crypto/streams/BlockedOutputStream.java
+++ b/core/src/main/java/org/apache/accumulo/core/crypto/streams/BlockedOutputStream.java
@@ -55,7 +55,7 @@
   @Override
   public synchronized void flush() throws IOException {
     if (!bb.hasArray()) {
-      throw new RuntimeException("BlockedOutputStream has no backing array.");
+      throw new IllegalStateException("BlockedOutputStream has no backing array.");
     }
     int size = bb.position();
     if (size == 0) {
diff --git a/core/src/main/java/org/apache/accumulo/core/data/ConditionalMutation.java b/core/src/main/java/org/apache/accumulo/core/data/ConditionalMutation.java
index 88fb8b5..4db8ca8 100644
--- a/core/src/main/java/org/apache/accumulo/core/data/ConditionalMutation.java
+++ b/core/src/main/java/org/apache/accumulo/core/data/ConditionalMutation.java
@@ -19,13 +19,14 @@
 package org.apache.accumulo.core.data;
 
 import static com.google.common.base.Preconditions.checkArgument;
+import static java.nio.charset.StandardCharsets.UTF_8;
 
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
-import java.util.Set;
 
+import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.hadoop.io.Text;
 
 /**
@@ -83,6 +84,50 @@
     return Collections.unmodifiableList(conditions);
   }
 
+  private String toString(ByteSequence bs) {
+    if (bs == null) {
+      return null;
+    }
+    return new String(bs.toArray(), UTF_8);
+  }
+
+  @Override
+  public String prettyPrint() {
+    StringBuilder sb = new StringBuilder(super.prettyPrint());
+    for (Condition c : conditions) {
+      sb.append(" condition: ");
+      sb.append(toString(c.getFamily()));
+      sb.append(":");
+      sb.append(toString(c.getQualifier()));
+      if (c.getValue() != null && !toString(c.getValue()).isBlank()) {
+        sb.append(" value: ");
+        sb.append(toString(c.getValue()));
+      }
+      if (c.getVisibility() != null && !toString(c.getVisibility()).isBlank()) {
+        sb.append(" visibility: '");
+        sb.append(toString(c.getVisibility()));
+        sb.append("'");
+      }
+      if (c.getTimestamp() != null) {
+        sb.append(" timestamp: ");
+        sb.append("'");
+        sb.append(c.getTimestamp());
+        sb.append("'");
+      }
+      if (c.getIterators().length != 0) {
+        sb.append(" iterator: ");
+        IteratorSetting[] iterators = c.getIterators();
+        for (IteratorSetting its : iterators) {
+          sb.append("'");
+          sb.append(its.toString());
+          sb.append("' ");
+        }
+      }
+      sb.append("\n");
+    }
+    return sb.toString();
+  }
+
   @Override
   public boolean equals(Object o) {
     if (o == this) {
@@ -105,10 +150,4 @@
     return result;
   }
 
-  @Deprecated
-  @Override
-  public void setReplicationSources(Set<String> sources) {
-    throw new UnsupportedOperationException(
-        "Conditional Mutations are not supported for replication");
-  }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/data/InstanceId.java b/core/src/main/java/org/apache/accumulo/core/data/InstanceId.java
index 598d359..70e9019 100644
--- a/core/src/main/java/org/apache/accumulo/core/data/InstanceId.java
+++ b/core/src/main/java/org/apache/accumulo/core/data/InstanceId.java
@@ -20,10 +20,9 @@
 
 import java.util.Objects;
 import java.util.UUID;
-import java.util.concurrent.ExecutionException;
 
-import com.google.common.cache.Cache;
-import com.google.common.cache.CacheBuilder;
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
 
 /**
  * A strongly typed representation of an Accumulo instance ID. The constructor for this class will
@@ -36,7 +35,7 @@
   // cache is for canonicalization/deduplication of created objects,
   // to limit the number of InstanceId objects in the JVM at any given moment
   // WeakReferences are used because we don't need them to stick around any longer than they need to
-  static final Cache<String,InstanceId> cache = CacheBuilder.newBuilder().weakValues().build();
+  static final Cache<String,InstanceId> cache = Caffeine.newBuilder().weakValues().build();
 
   private InstanceId(String canonical) {
     super(canonical);
@@ -49,12 +48,7 @@
    * @return InstanceId object
    */
   public static InstanceId of(final String canonical) {
-    try {
-      return cache.get(canonical, () -> new InstanceId(canonical));
-    } catch (ExecutionException e) {
-      throw new AssertionError(
-          "This should never happen: ID constructor should never return null.");
-    }
+    return cache.get(canonical, k -> new InstanceId(canonical));
   }
 
   /**
diff --git a/core/src/main/java/org/apache/accumulo/core/data/LoadPlan.java b/core/src/main/java/org/apache/accumulo/core/data/LoadPlan.java
index aecd25f..311042d 100644
--- a/core/src/main/java/org/apache/accumulo/core/data/LoadPlan.java
+++ b/core/src/main/java/org/apache/accumulo/core/data/LoadPlan.java
@@ -125,7 +125,7 @@
               "Start row is greater than or equal to end row : " + srs + " " + ers);
         }
       } else {
-        throw new RuntimeException();
+        throw new IllegalStateException();
       }
 
     }
diff --git a/core/src/main/java/org/apache/accumulo/core/data/Mutation.java b/core/src/main/java/org/apache/accumulo/core/data/Mutation.java
index 53f834c..b8c6c22 100644
--- a/core/src/main/java/org/apache/accumulo/core/data/Mutation.java
+++ b/core/src/main/java/org/apache/accumulo/core/data/Mutation.java
@@ -19,7 +19,6 @@
 package org.apache.accumulo.core.data;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
-import static java.util.Objects.requireNonNull;
 
 import java.io.DataInput;
 import java.io.DataOutput;
@@ -28,9 +27,7 @@
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
-import java.util.HashSet;
 import java.util.List;
-import java.util.Set;
 
 import org.apache.accumulo.core.dataImpl.thrift.TMutation;
 import org.apache.accumulo.core.security.ColumnVisibility;
@@ -109,9 +106,6 @@
 
   private List<ColumnUpdate> updates;
 
-  private static final Set<String> EMPTY = Collections.emptySet();
-  private Set<String> replicationSources = EMPTY;
-
   private static final byte[] EMPTY_BYTES = new byte[0];
 
   private void serialize() {
@@ -242,10 +236,6 @@
     this.entries = tmutation.entries;
     this.values = ByteBufferUtil.toBytesList(tmutation.values);
 
-    if (tmutation.isSetSources()) {
-      this.replicationSources = new HashSet<>(tmutation.sources);
-    }
-
     if (this.row == null) {
       throw new IllegalArgumentException("null row");
     }
@@ -265,7 +255,6 @@
     this.data = m.data;
     this.entries = m.entries;
     this.values = m.values;
-    this.replicationSources = m.replicationSources;
   }
 
   /**
@@ -1360,58 +1349,6 @@
     return entries;
   }
 
-  /**
-   * Add a new element to the set of peers which this Mutation originated from
-   *
-   * @param peer the peer to add
-   * @since 1.7.0
-   * @deprecated The feature pertaining to this method was deprecated in 2.1.0, but this method was
-   *             overlooked when annotating the code. It is being marked as deprecated in 2.1.1 in
-   *             order to correct that oversight, and will be removed in 3.0.0 with the rest of the
-   *             code pertaining to this feature.
-   */
-  @Deprecated(since = "2.1.1")
-  public void addReplicationSource(String peer) {
-    if (replicationSources == null || replicationSources == EMPTY) {
-      replicationSources = new HashSet<>();
-    }
-
-    replicationSources.add(peer);
-  }
-
-  /**
-   * Set the replication peers which this Mutation originated from
-   *
-   * @param sources Set of peer names which have processed this update
-   * @since 1.7.0
-   * @deprecated The feature pertaining to this method was deprecated in 2.1.0, but this method was
-   *             overlooked when annotating the code. It is being marked as deprecated in 2.1.1 in
-   *             order to correct that oversight, and will be removed in 3.0.0 with the rest of the
-   *             code pertaining to this feature.
-   */
-  @Deprecated(since = "2.1.1")
-  public void setReplicationSources(Set<String> sources) {
-    requireNonNull(sources);
-    this.replicationSources = sources;
-  }
-
-  /**
-   * Return the replication sources for this Mutation
-   *
-   * @return An unmodifiable view of the replication sources
-   * @deprecated The feature pertaining to this method was deprecated in 2.1.0, but this method was
-   *             overlooked when annotating the code. It is being marked as deprecated in 2.1.1 in
-   *             order to correct that oversight, and will be removed in 3.0.0 with the rest of the
-   *             code pertaining to this feature.
-   */
-  @Deprecated(since = "2.1.1")
-  public Set<String> getReplicationSources() {
-    if (replicationSources == null) {
-      return EMPTY;
-    }
-    return Collections.unmodifiableSet(replicationSources);
-  }
-
   @Override
   public void readFields(DataInput in) throws IOException {
 
@@ -1454,9 +1391,9 @@
 
     if ((first & 0x02) == 0x02) {
       int numMutations = WritableUtils.readVInt(in);
-      this.replicationSources = new HashSet<>();
       for (int i = 0; i < numMutations; i++) {
-        replicationSources.add(WritableUtils.readString(in));
+        // consume the replication sources that may have been previously serialized
+        WritableUtils.readString(in);
       }
     }
   }
@@ -1530,10 +1467,9 @@
     final byte[] integerBuffer = new byte[5];
     serialize();
     byte hasValues = (values == null) ? 0 : (byte) 1;
-    if (!replicationSources.isEmpty()) {
-      // Use 2nd least-significant bit for whether or not we have replication sources
-      hasValues = (byte) (0x02 | hasValues);
-    }
+    // When replication sources were supported, we used the 2nd least-significant bit to denote
+    // their presence, but this is no longer used; kept here for historical explanation only
+    // hasValues = (byte) (0x02 | hasValues);
     out.write((byte) (0x80 | hasValues));
 
     UnsynchronizedBuffer.writeVInt(out, integerBuffer, row.length);
@@ -1550,12 +1486,6 @@
         out.write(val);
       }
     }
-    if ((0x02 & hasValues) == 0x02) {
-      UnsynchronizedBuffer.writeVInt(out, integerBuffer, replicationSources.size());
-      for (String source : replicationSources) {
-        WritableUtils.writeString(out, source);
-      }
-    }
   }
 
   @Override
@@ -1591,9 +1521,6 @@
     ByteBuffer otherData = m.serializedSnapshot();
     if (Arrays.equals(row, m.row) && entries == m.entries && myData.equals(otherData)) {
       // If two mutations don't have the same
-      if (!replicationSources.equals(m.replicationSources)) {
-        return false;
-      }
       if (values == null && m.values == null) {
         return true;
       }
@@ -1631,12 +1558,7 @@
       this.serialize();
     }
     ByteBuffer data = serializedSnapshot();
-    TMutation tmutation =
-        new TMutation(ByteBuffer.wrap(row), data, ByteBufferUtil.toByteBuffers(values), entries);
-    if (!this.replicationSources.isEmpty()) {
-      tmutation.setSources(new ArrayList<>(replicationSources));
-    }
-    return tmutation;
+    return new TMutation(ByteBuffer.wrap(row), data, ByteBufferUtil.toByteBuffers(values), entries);
   }
 
   /**
diff --git a/core/src/main/java/org/apache/accumulo/core/data/NamespaceId.java b/core/src/main/java/org/apache/accumulo/core/data/NamespaceId.java
index 90a8c74..c201113 100644
--- a/core/src/main/java/org/apache/accumulo/core/data/NamespaceId.java
+++ b/core/src/main/java/org/apache/accumulo/core/data/NamespaceId.java
@@ -18,10 +18,8 @@
  */
 package org.apache.accumulo.core.data;
 
-import java.util.concurrent.ExecutionException;
-
-import com.google.common.cache.Cache;
-import com.google.common.cache.CacheBuilder;
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
 
 /**
  * A strongly typed representation of a namespace ID. This class cannot be used to get a namespace
@@ -35,7 +33,7 @@
   // cache is for canonicalization/deduplication of created objects,
   // to limit the number of NamespaceId objects in the JVM at any given moment
   // WeakReferences are used because we don't need them to stick around any longer than they need to
-  static final Cache<String,NamespaceId> cache = CacheBuilder.newBuilder().weakValues().build();
+  static final Cache<String,NamespaceId> cache = Caffeine.newBuilder().weakValues().build();
 
   private NamespaceId(String canonical) {
     super(canonical);
@@ -48,11 +46,6 @@
    * @return NamespaceId object
    */
   public static NamespaceId of(final String canonical) {
-    try {
-      return cache.get(canonical, () -> new NamespaceId(canonical));
-    } catch (ExecutionException e) {
-      throw new AssertionError(
-          "This should never happen: ID constructor should never return null.");
-    }
+    return cache.get(canonical, k -> new NamespaceId(canonical));
   }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/data/TableId.java b/core/src/main/java/org/apache/accumulo/core/data/TableId.java
index 9493d8d..a97c9d2 100644
--- a/core/src/main/java/org/apache/accumulo/core/data/TableId.java
+++ b/core/src/main/java/org/apache/accumulo/core/data/TableId.java
@@ -18,10 +18,8 @@
  */
 package org.apache.accumulo.core.data;
 
-import java.util.concurrent.ExecutionException;
-
-import com.google.common.cache.Cache;
-import com.google.common.cache.CacheBuilder;
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
 
 /**
  * A strongly typed representation of a table ID. This class cannot be used to get a table ID from a
@@ -35,7 +33,7 @@
   // cache is for canonicalization/deduplication of created objects,
   // to limit the number of TableId objects in the JVM at any given moment
   // WeakReferences are used because we don't need them to stick around any longer than they need to
-  static final Cache<String,TableId> cache = CacheBuilder.newBuilder().weakValues().build();
+  static final Cache<String,TableId> cache = Caffeine.newBuilder().weakValues().build();
 
   private TableId(final String canonical) {
     super(canonical);
@@ -48,11 +46,6 @@
    * @return TableId object
    */
   public static TableId of(final String canonical) {
-    try {
-      return cache.get(canonical, () -> new TableId(canonical));
-    } catch (ExecutionException e) {
-      throw new AssertionError(
-          "This should never happen: ID constructor should never return null.");
-    }
+    return cache.get(canonical, k -> new TableId(canonical));
   }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/data/TabletId.java b/core/src/main/java/org/apache/accumulo/core/data/TabletId.java
index 51bfa1a..2d676d1 100644
--- a/core/src/main/java/org/apache/accumulo/core/data/TabletId.java
+++ b/core/src/main/java/org/apache/accumulo/core/data/TabletId.java
@@ -31,12 +31,6 @@
    */
   TableId getTable();
 
-  /**
-   * @deprecated use {@link #getTable()} and {@link TableId#canonical()} instead
-   */
-  @Deprecated(since = "2.1.0")
-  Text getTableId();
-
   Text getEndRow();
 
   Text getPrevEndRow();
diff --git a/core/src/main/java/org/apache/accumulo/core/data/constraints/VisibilityConstraint.java b/core/src/main/java/org/apache/accumulo/core/data/constraints/VisibilityConstraint.java
index 62531c5..87b60df 100644
--- a/core/src/main/java/org/apache/accumulo/core/data/constraints/VisibilityConstraint.java
+++ b/core/src/main/java/org/apache/accumulo/core/data/constraints/VisibilityConstraint.java
@@ -24,12 +24,11 @@
 import java.util.HashSet;
 import java.util.List;
 
+import org.apache.accumulo.access.AccessEvaluator;
+import org.apache.accumulo.access.InvalidAccessExpressionException;
+import org.apache.accumulo.core.data.ArrayByteSequence;
 import org.apache.accumulo.core.data.ColumnUpdate;
 import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.security.ColumnVisibility;
-import org.apache.accumulo.core.security.VisibilityEvaluator;
-import org.apache.accumulo.core.security.VisibilityParseException;
-import org.apache.accumulo.core.util.BadArgumentException;
 
 /**
  * A constraint that checks the visibility of columns against the actor's authorizations. Violation
@@ -64,7 +63,7 @@
       ok = new HashSet<>();
     }
 
-    VisibilityEvaluator ve = null;
+    AccessEvaluator ve = null;
 
     for (ColumnUpdate update : updates) {
 
@@ -78,14 +77,15 @@
         try {
 
           if (ve == null) {
-            ve = new VisibilityEvaluator(env.getAuthorizationsContainer());
+            var authContainer = env.getAuthorizationsContainer();
+            ve = AccessEvaluator.of(auth -> authContainer.contains(new ArrayByteSequence(auth)));
           }
 
-          if (!ve.evaluate(new ColumnVisibility(cv))) {
+          if (!ve.canAccess(cv)) {
             return Collections.singletonList((short) 2);
           }
 
-        } catch (BadArgumentException | VisibilityParseException bae) {
+        } catch (InvalidAccessExpressionException iaee) {
           return Collections.singletonList((short) 1);
         }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/dataImpl/KeyExtent.java b/core/src/main/java/org/apache/accumulo/core/dataImpl/KeyExtent.java
index c0a40dd..1069a5e 100644
--- a/core/src/main/java/org/apache/accumulo/core/dataImpl/KeyExtent.java
+++ b/core/src/main/java/org/apache/accumulo/core/dataImpl/KeyExtent.java
@@ -20,6 +20,8 @@
 
 import static java.nio.charset.StandardCharsets.UTF_8;
 import static java.util.Objects.requireNonNull;
+import static org.apache.accumulo.core.util.RowRangeUtil.requireKeyExtentDataRange;
+import static org.apache.accumulo.core.util.RowRangeUtil.stripZeroTail;
 
 import java.io.ByteArrayOutputStream;
 import java.io.DataInput;
@@ -46,8 +48,7 @@
 import org.apache.accumulo.core.data.TabletId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.dataImpl.thrift.TKeyExtent;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.TabletColumnFamily;
 import org.apache.accumulo.core.util.ByteBufferUtil;
@@ -389,6 +390,65 @@
     return new Range(metadataPrevRow, prevEndRow() == null, toMetaRow(), true);
   }
 
+  /**
+   * Creates a KeyExtent which represents the intersection of this KeyExtent and the passed in
+   * range.
+   * <p>
+   * <b>Note:</b> The range provided must be a range that is derived from a KeyExtent. This means
+   * the range must be in the format of a row range and also requires an exclusive start key, which
+   * is the format that {@link #toDataRange()} uses
+   *
+   * @param range range to clip to
+   * @return the intersection of this KeyExtent and the given range
+   * @throws IllegalArgumentException if the KeyExtent and range do not overlap
+   */
+  public KeyExtent clip(Range range) {
+    return clip(range, false);
+  }
+
+  /**
+   * Creates a KeyExtent which represents the intersection of this KeyExtent and the passed in
+   * range. Unlike {@link #clip(Range)}, this method can optionally return null if the given range
+   * and this KeyExtent do not overlap, instead of throwing an exception. The returnNullIfDisjoint
+   * parameter controls this behavior.
+   * <p>
+   * <b>Note:</b> The range provided must be a range that is derived from a KeyExtent. This means
+   * the range must be in the format of a row range and also requires an exclusive start key, which
+   * is the format that {@link #toDataRange()} uses
+   *
+   * @param range range to clip to
+   * @param returnNullIfDisjoint true to return null if ranges are disjoint, false to throw an
+   *        exception
+   * @return the intersection of this KeyExtent and the given range, or null if given range and this
+   *         KeyExtent do not overlap and returnNullIfDisjoint is true
+   * @throws IllegalArgumentException if the KeyExtent and range does not overlap and
+   *         returnNullIfDisjoint is false
+   *
+   * @see KeyExtent#clip(Range)
+   **/
+  public KeyExtent clip(Range range, boolean returnNullIfDisjoint) {
+    // This will require a range that matches a row range generated by toDataRange()
+    // This range itself will be required to be an inclusive start and exclusive end
+    // The start and end rows will be required to be exclusive keys (ending in 0x00)
+    requireKeyExtentDataRange(range);
+
+    // If returnNullIfDisjoint is false then this will throw an exception if
+    // the ranges are disjoint, otherwise we can just return null
+    final Range clippedRange = this.toDataRange().clip(range, returnNullIfDisjoint);
+    if (clippedRange == null) {
+      return null;
+    }
+
+    // Build the new KeyExtent with the clipped range. We need to strip off the ending byte
+    // which will essentially reverse what toDataRange() does
+    Text endRow = clippedRange.getEndKey() != null
+        ? new Text(stripZeroTail(clippedRange.getEndKey().getRowData()).toArray()) : null;
+    Text prevEndRow = clippedRange.getStartKey() != null
+        ? new Text(stripZeroTail(clippedRange.getStartKey().getRowData()).toArray()) : null;
+
+    return new KeyExtent(tableId, endRow, prevEndRow);
+  }
+
   private boolean startsAfter(KeyExtent other) {
     KeyExtent nke = requireNonNull(other);
     return tableId().compareTo(nke.tableId()) > 0 || (prevEndRow() != null && nke.endRow() != null
@@ -475,12 +535,16 @@
     return prevExtent.endRow().equals(prevEndRow());
   }
 
+  public boolean isSystemTable() {
+    return AccumuloTable.allTableIds().contains(tableId());
+  }
+
   public boolean isMeta() {
-    return tableId().equals(MetadataTable.ID) || isRootTablet();
+    return tableId().equals(AccumuloTable.METADATA.tableId()) || isRootTablet();
   }
 
   public boolean isRootTablet() {
-    return tableId().equals(RootTable.ID);
+    return tableId().equals(AccumuloTable.ROOT.tableId());
   }
 
   public String obscured() {
@@ -488,7 +552,7 @@
     try {
       digester = MessageDigest.getInstance(OBSCURING_HASH_ALGORITHM);
     } catch (NoSuchAlgorithmException e) {
-      throw new RuntimeException(e);
+      throw new IllegalStateException(e);
     }
     if (endRow() != null && endRow().getLength() > 0) {
       digester.update(endRow().getBytes(), 0, endRow().getLength());
diff --git a/core/src/main/java/org/apache/accumulo/core/dataImpl/TabletIdImpl.java b/core/src/main/java/org/apache/accumulo/core/dataImpl/TabletIdImpl.java
index e5a16dd..4700980 100644
--- a/core/src/main/java/org/apache/accumulo/core/dataImpl/TabletIdImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/dataImpl/TabletIdImpl.java
@@ -42,12 +42,6 @@
   }
 
   @Override
-  @Deprecated(since = "2.1.0")
-  public Text getTableId() {
-    return new Text(ke.tableId().canonical());
-  }
-
-  @Override
   public Text getEndRow() {
     return ke.endRow();
   }
diff --git a/core/src/main/java/org/apache/accumulo/core/fate/AdminUtil.java b/core/src/main/java/org/apache/accumulo/core/fate/AdminUtil.java
index 3ea322e..78f107f 100644
--- a/core/src/main/java/org/apache/accumulo/core/fate/AdminUtil.java
+++ b/core/src/main/java/org/apache/accumulo/core/fate/AdminUtil.java
@@ -20,6 +20,7 @@
 
 import static java.nio.charset.StandardCharsets.UTF_8;
 
+import java.time.Duration;
 import java.time.ZoneOffset;
 import java.time.format.DateTimeFormatter;
 import java.util.ArrayList;
@@ -32,16 +33,15 @@
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.fate.ReadOnlyTStore.TStatus;
 import org.apache.accumulo.core.fate.zookeeper.FateLock;
 import org.apache.accumulo.core.fate.zookeeper.FateLock.FateLockPath;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock.ServiceLockPath;
 import org.apache.accumulo.core.fate.zookeeper.ZooReader;
 import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeMissingPolicy;
+import org.apache.accumulo.core.lock.ServiceLock;
+import org.apache.accumulo.core.lock.ServiceLock.ServiceLockPath;
 import org.apache.accumulo.core.util.FastFormat;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
@@ -368,7 +368,7 @@
 
         long timeCreated = zs.timeCreated(tid);
 
-        zs.unreserve(tid, 0, TimeUnit.MILLISECONDS);
+        zs.unreserve(tid, Duration.ZERO);
 
         if (includeByStatus(status, filterStatus) && includeByTxid(tid, filterTxid)) {
           statuses
@@ -471,7 +471,7 @@
         break;
     }
 
-    zs.unreserve(txid, 0, TimeUnit.MILLISECONDS);
+    zs.unreserve(txid, Duration.ZERO);
     return state;
   }
 
@@ -515,7 +515,7 @@
         break;
     }
 
-    zs.unreserve(txid, 0, TimeUnit.MILLISECONDS);
+    zs.unreserve(txid, Duration.ZERO);
     return state;
   }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/fate/AgeOffStore.java b/core/src/main/java/org/apache/accumulo/core/fate/AgeOffStore.java
index ca016d0..bd2bd52 100644
--- a/core/src/main/java/org/apache/accumulo/core/fate/AgeOffStore.java
+++ b/core/src/main/java/org/apache/accumulo/core/fate/AgeOffStore.java
@@ -19,13 +19,13 @@
 package org.apache.accumulo.core.fate;
 
 import java.io.Serializable;
+import java.time.Duration;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
-import java.util.concurrent.TimeUnit;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -108,7 +108,7 @@
           }
 
         } finally {
-          store.unreserve(txid, 0, TimeUnit.MILLISECONDS);
+          store.unreserve(txid, Duration.ZERO);
         }
       } catch (Exception e) {
         log.warn("Failed to age off FATE tx " + FateTxId.formatTid(txid), e);
@@ -138,7 +138,7 @@
             break;
         }
       } finally {
-        store.unreserve(txid, 0, TimeUnit.MILLISECONDS);
+        store.unreserve(txid, Duration.ZERO);
       }
     }
   }
@@ -166,8 +166,8 @@
   }
 
   @Override
-  public void unreserve(long tid, long deferTime, TimeUnit deferTimeUnit) {
-    store.unreserve(tid, deferTime, deferTimeUnit);
+  public void unreserve(long tid, Duration deferTime) {
+    store.unreserve(tid, deferTime);
   }
 
   @Override
diff --git a/core/src/main/java/org/apache/accumulo/core/fate/Fate.java b/core/src/main/java/org/apache/accumulo/core/fate/Fate.java
index 4eb690e..83793ec 100644
--- a/core/src/main/java/org/apache/accumulo/core/fate/Fate.java
+++ b/core/src/main/java/org/apache/accumulo/core/fate/Fate.java
@@ -18,6 +18,7 @@
  */
 package org.apache.accumulo.core.fate;
 
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 import static java.util.concurrent.TimeUnit.MINUTES;
 import static java.util.concurrent.TimeUnit.SECONDS;
 import static org.apache.accumulo.core.fate.ReadOnlyTStore.TStatus.FAILED;
@@ -29,12 +30,12 @@
 import static org.apache.accumulo.core.fate.ReadOnlyTStore.TStatus.UNKNOWN;
 import static org.apache.accumulo.core.util.ShutdownUtil.isIOException;
 
+import java.time.Duration;
 import java.util.EnumSet;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.RejectedExecutionException;
 import java.util.concurrent.ScheduledThreadPoolExecutor;
 import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.function.Function;
 
@@ -60,8 +61,8 @@
 
   private final TStore<T> store;
   private final T environment;
-  private ScheduledThreadPoolExecutor fatePoolWatcher;
-  private ExecutorService executor;
+  private final ScheduledThreadPoolExecutor fatePoolWatcher;
+  private final ExecutorService executor;
 
   private static final EnumSet<TStatus> FINISHED_STATES = EnumSet.of(FAILED, SUCCESSFUL, UNKNOWN);
 
@@ -132,7 +133,7 @@
           runnerLog.error("Uncaught exception in FATE runner thread.", e);
         } finally {
           if (tid != null) {
-            store.unreserve(tid, deferTime, TimeUnit.MILLISECONDS);
+            store.unreserve(tid, Duration.ofMillis(deferTime));
           }
         }
       }
@@ -161,7 +162,7 @@
         while (true) {
           // Nothing is going to work well at this point, so why even try. Just wait for the end,
           // preventing this FATE thread from processing further work and likely failing.
-          UtilWaitThread.sleepUninterruptibly(1, MINUTES);
+          sleepUninterruptibly(1, MINUTES);
         }
       }
     }
@@ -227,24 +228,16 @@
 
   /**
    * Creates a Fault-tolerant executor.
-   * <p>
-   * Note: Users of this class should call {@link #startTransactionRunners(AccumuloConfiguration)}
-   * to launch the worker threads after creating a Fate object.
    *
    * @param toLogStrFunc A function that converts Repo to Strings that are suitable for logging
    */
-  public Fate(T environment, TStore<T> store, Function<Repo<T>,String> toLogStrFunc) {
+  public Fate(T environment, TStore<T> store, Function<Repo<T>,String> toLogStrFunc,
+      AccumuloConfiguration conf) {
     this.store = FateLogger.wrap(store, toLogStrFunc);
     this.environment = environment;
-  }
-
-  /**
-   * Launches the specified number of worker threads.
-   */
-  public void startTransactionRunners(AccumuloConfiguration conf) {
     final ThreadPoolExecutor pool = ThreadPools.getServerThreadPools().createExecutorService(conf,
         Property.MANAGER_FATE_THREADPOOL_SIZE, true);
-    fatePoolWatcher =
+    this.fatePoolWatcher =
         ThreadPools.getServerThreadPools().createGeneralScheduledExecutorService(conf);
     ThreadPools.watchCriticalScheduledTask(fatePoolWatcher.schedule(() -> {
       // resize the pool if the property changed
@@ -269,7 +262,7 @@
         }
       }
     }, 3, SECONDS));
-    executor = pool;
+    this.executor = pool;
   }
 
   // get a transaction id back to the requester before doing any work
@@ -290,7 +283,7 @@
             store.push(tid, repo);
           } catch (StackOverflowException e) {
             // this should not happen
-            throw new RuntimeException(e);
+            throw new IllegalStateException(e);
           }
         }
 
@@ -303,7 +296,7 @@
         store.setStatus(tid, SUBMITTED);
       }
     } finally {
-      store.unreserve(tid, 0, TimeUnit.MILLISECONDS);
+      store.unreserve(tid, Duration.ZERO);
     }
 
   }
@@ -339,7 +332,7 @@
             return false;
           }
         } finally {
-          store.unreserve(tid, 0, TimeUnit.MILLISECONDS);
+          store.unreserve(tid, Duration.ZERO);
         }
       } else {
         // reserved, lets retry.
@@ -370,7 +363,7 @@
           break;
       }
     } finally {
-      store.unreserve(tid, 0, TimeUnit.MILLISECONDS);
+      store.unreserve(tid, Duration.ZERO);
     }
   }
 
@@ -383,7 +376,7 @@
       }
       return (String) store.getTransactionInfo(tid, TxInfo.RETURN_VALUE);
     } finally {
-      store.unreserve(tid, 0, TimeUnit.MILLISECONDS);
+      store.unreserve(tid, Duration.ZERO);
     }
   }
 
@@ -397,7 +390,7 @@
       }
       return (Exception) store.getTransactionInfo(tid, TxInfo.EXCEPTION);
     } finally {
-      store.unreserve(tid, 0, TimeUnit.MILLISECONDS);
+      store.unreserve(tid, Duration.ZERO);
     }
   }
 
@@ -407,7 +400,9 @@
   public void shutdown() {
     keepRunning.set(false);
     fatePoolWatcher.shutdown();
-    executor.shutdown();
+    if (executor != null) {
+      executor.shutdown();
+    }
   }
 
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/fate/ReadOnlyTStore.java b/core/src/main/java/org/apache/accumulo/core/fate/ReadOnlyTStore.java
index 4a216f1..0b48c3b 100644
--- a/core/src/main/java/org/apache/accumulo/core/fate/ReadOnlyTStore.java
+++ b/core/src/main/java/org/apache/accumulo/core/fate/ReadOnlyTStore.java
@@ -19,9 +19,9 @@
 package org.apache.accumulo.core.fate;
 
 import java.io.Serializable;
+import java.time.Duration;
 import java.util.EnumSet;
 import java.util.List;
-import java.util.concurrent.TimeUnit;
 
 /**
  * Read only access to a Transaction Store.
@@ -79,9 +79,8 @@
    * @param tid transaction id, previously reserved.
    * @param deferTime time to keep this transaction out of the pool used in the {@link #reserve()
    *        reserve} method. must be non-negative.
-   * @param deferTimeUnit the time unit of deferTime
    */
-  void unreserve(long tid, long deferTime, TimeUnit deferTimeUnit);
+  void unreserve(long tid, Duration deferTime);
 
   /**
    * Get the current operation for the given transaction id.
diff --git a/core/src/main/java/org/apache/accumulo/core/fate/ZooStore.java b/core/src/main/java/org/apache/accumulo/core/fate/ZooStore.java
index ff7a297..c3de5f2 100644
--- a/core/src/main/java/org/apache/accumulo/core/fate/ZooStore.java
+++ b/core/src/main/java/org/apache/accumulo/core/fate/ZooStore.java
@@ -18,9 +18,10 @@
  */
 package org.apache.accumulo.core.fate;
 
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 import static java.nio.charset.StandardCharsets.UTF_8;
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
@@ -28,7 +29,8 @@
 import java.io.ObjectInputStream;
 import java.io.ObjectOutputStream;
 import java.io.Serializable;
-import java.security.SecureRandom;
+import java.io.UncheckedIOException;
+import java.time.Duration;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.EnumSet;
@@ -37,12 +39,12 @@
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeMissingPolicy;
 import org.apache.accumulo.core.util.FastFormat;
+import org.apache.accumulo.core.util.time.NanoTime;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.KeeperException.NoNodeException;
 import org.apache.zookeeper.KeeperException.NodeExistsException;
@@ -62,8 +64,7 @@
   private ZooReaderWriter zk;
   private String lastReserved = "";
   private Set<Long> reserved;
-  private Map<Long,Long> deferred;
-  private static final SecureRandom random = new SecureRandom();
+  private Map<Long,NanoTime> deferred;
   private long statusChangeEvents = 0;
   private int reservationsWaiting = 0;
 
@@ -74,7 +75,7 @@
       oos.writeObject(o);
       return baos.toByteArray();
     } catch (IOException e) {
-      throw new RuntimeException(e);
+      throw new UncheckedIOException(e);
     }
   }
 
@@ -85,8 +86,10 @@
     try (ByteArrayInputStream bais = new ByteArrayInputStream(ser);
         ObjectInputStream ois = new ObjectInputStream(bais)) {
       return ois.readObject();
-    } catch (Exception e) {
-      throw new RuntimeException(e);
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    } catch (ReflectiveOperationException e) {
+      throw new IllegalStateException(e);
     }
   }
 
@@ -118,14 +121,14 @@
     while (true) {
       try {
         // looking at the code for SecureRandom, it appears to be thread safe
-        long tid = random.nextLong() & 0x7fffffffffffffffL;
+        long tid = RANDOM.get().nextLong() & 0x7fffffffffffffffL;
         zk.putPersistentData(getTXPath(tid), TStatus.NEW.name().getBytes(UTF_8),
             NodeExistsPolicy.FAIL);
         return tid;
       } catch (NodeExistsException nee) {
         // exist, so just try another random #
-      } catch (Exception e) {
-        throw new RuntimeException(e);
+      } catch (KeeperException | InterruptedException e) {
+        throw new IllegalStateException(e);
       }
     }
   }
@@ -161,7 +164,7 @@
             }
 
             if (deferred.containsKey(tid)) {
-              if ((deferred.get(tid) - System.nanoTime()) < 0) {
+              if (deferred.get(tid).elapsed().compareTo(Duration.ZERO) > 0) {
                 deferred.remove(tid);
               } else {
                 continue;
@@ -188,7 +191,7 @@
           } catch (NoNodeException nne) {
             // node deleted after we got the list of children, its ok
             unreserve(tid);
-          } catch (Exception e) {
+          } catch (KeeperException | InterruptedException | RuntimeException e) {
             unreserve(tid);
             throw e;
           }
@@ -200,19 +203,18 @@
             if (deferred.isEmpty()) {
               this.wait(5000);
             } else {
-              long currTime = System.nanoTime();
-              long minWait =
-                  deferred.values().stream().mapToLong(l -> l - currTime).min().getAsLong();
-              long waitTime = TimeUnit.MILLISECONDS.convert(minWait, TimeUnit.NANOSECONDS);
-              if (waitTime > 0) {
-                this.wait(Math.min(waitTime, 5000));
+              var now = NanoTime.now();
+              long minWait = deferred.values().stream()
+                  .mapToLong(nanoTime -> nanoTime.subtract(now).toMillis()).min().orElseThrow();
+              if (minWait > 0) {
+                this.wait(Math.min(minWait, 5000));
               }
             }
           }
         }
       }
-    } catch (InterruptedException | KeeperException e) {
-      throw new RuntimeException(e);
+    } catch (KeeperException | InterruptedException e) {
+      throw new IllegalStateException(e);
     }
   }
 
@@ -225,7 +227,7 @@
           try {
             this.wait(1000);
           } catch (InterruptedException e) {
-            throw new RuntimeException(e);
+            throw new IllegalStateException(e);
           }
         }
 
@@ -270,10 +272,9 @@
   }
 
   @Override
-  public void unreserve(long tid, long deferTime, TimeUnit deferTimeUnit) {
-    deferTime = TimeUnit.NANOSECONDS.convert(deferTime, deferTimeUnit);
+  public void unreserve(long tid, Duration deferTime) {
 
-    if (deferTime < 0) {
+    if (deferTime.isNegative()) {
       throw new IllegalArgumentException("deferTime < 0 : " + deferTime);
     }
 
@@ -283,8 +284,8 @@
             "Tried to unreserve id that was not reserved " + FateTxId.formatTid(tid));
       }
 
-      if (deferTime > 0) {
-        deferred.put(tid, System.nanoTime() + deferTime);
+      if (deferTime.compareTo(Duration.ZERO) > 0) {
+        deferred.put(tid, NanoTime.nowPlus(deferTime));
       }
 
       this.notifyAll();
@@ -317,7 +318,7 @@
             return null;
           }
         } catch (KeeperException.NoNodeException ex) {
-          throw new RuntimeException(ex);
+          throw new IllegalStateException(ex);
         }
 
         byte[] ser = zk.getData(txpath + "/" + top);
@@ -328,8 +329,8 @@
         log.debug("zookeeper error reading " + txpath + ": " + ex, ex);
         sleepUninterruptibly(100, MILLISECONDS);
         continue;
-      } catch (Exception e) {
-        throw new RuntimeException(e);
+      } catch (KeeperException | InterruptedException e) {
+        throw new IllegalStateException(e);
       }
     }
     return null;
@@ -369,8 +370,8 @@
       zk.putPersistentSequential(txpath + "/repo_", serialize(repo));
     } catch (StackOverflowException soe) {
       throw soe;
-    } catch (Exception e) {
-      throw new RuntimeException(e);
+    } catch (KeeperException | InterruptedException e) {
+      throw new IllegalStateException(e);
     }
   }
 
@@ -385,8 +386,8 @@
         throw new IllegalStateException("Tried to pop when empty " + FateTxId.formatTid(tid));
       }
       zk.recursiveDelete(txpath + "/" + top, NodeMissingPolicy.SKIP);
-    } catch (Exception e) {
-      throw new RuntimeException(e);
+    } catch (KeeperException | InterruptedException e) {
+      throw new IllegalStateException(e);
     }
   }
 
@@ -395,8 +396,8 @@
       return TStatus.valueOf(new String(zk.getData(getTXPath(tid)), UTF_8));
     } catch (NoNodeException nne) {
       return TStatus.UNKNOWN;
-    } catch (Exception e) {
-      throw new RuntimeException(e);
+    } catch (KeeperException | InterruptedException e) {
+      throw new IllegalStateException(e);
     }
   }
 
@@ -425,7 +426,7 @@
           try {
             this.wait(5000);
           } catch (InterruptedException e) {
-            throw new RuntimeException(e);
+            throw new IllegalStateException(e);
           }
         }
       }
@@ -439,8 +440,8 @@
     try {
       zk.putPersistentData(getTXPath(tid), status.name().getBytes(UTF_8),
           NodeExistsPolicy.OVERWRITE);
-    } catch (Exception e) {
-      throw new RuntimeException(e);
+    } catch (KeeperException | InterruptedException e) {
+      throw new IllegalStateException(e);
     }
 
     synchronized (this) {
@@ -455,8 +456,8 @@
 
     try {
       zk.recursiveDelete(getTXPath(tid), NodeMissingPolicy.SKIP);
-    } catch (Exception e) {
-      throw new RuntimeException(e);
+    } catch (KeeperException | InterruptedException e) {
+      throw new IllegalStateException(e);
     }
   }
 
@@ -476,8 +477,8 @@
         data[1] = ' ';
         zk.putPersistentData(getTXPath(tid) + "/" + txInfo, data, NodeExistsPolicy.OVERWRITE);
       }
-    } catch (Exception e2) {
-      throw new RuntimeException(e2);
+    } catch (KeeperException | InterruptedException e2) {
+      throw new IllegalStateException(e2);
     }
   }
 
@@ -499,8 +500,8 @@
       }
     } catch (NoNodeException nne) {
       return null;
-    } catch (Exception e) {
-      throw new RuntimeException(e);
+    } catch (KeeperException | InterruptedException e) {
+      throw new IllegalStateException(e);
     }
   }
 
@@ -513,8 +514,8 @@
         l.add(parseTid(txid));
       }
       return l;
-    } catch (Exception e) {
-      throw new RuntimeException(e);
+    } catch (KeeperException | InterruptedException e) {
+      throw new IllegalStateException(e);
     }
   }
 
@@ -541,7 +542,7 @@
       } catch (KeeperException.NoNodeException e) {
         return Collections.emptyList();
       } catch (KeeperException | InterruptedException e1) {
-        throw new RuntimeException(e1);
+        throw new IllegalStateException(e1);
       }
 
       ops = new ArrayList<>(ops);
@@ -561,7 +562,7 @@
             // children changed so start over
             continue outer;
           } catch (KeeperException | InterruptedException e) {
-            throw new RuntimeException(e);
+            throw new IllegalStateException(e);
           }
         }
       }
diff --git a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/DistributedReadWriteLock.java b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/DistributedReadWriteLock.java
index 852a40b..0bf4af1 100644
--- a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/DistributedReadWriteLock.java
+++ b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/DistributedReadWriteLock.java
@@ -40,7 +40,7 @@
  */
 public class DistributedReadWriteLock implements java.util.concurrent.locks.ReadWriteLock {
 
-  static enum LockType {
+  public enum LockType {
     READ, WRITE,
   }
 
@@ -107,7 +107,11 @@
 
   private static final Logger log = LoggerFactory.getLogger(DistributedReadWriteLock.class);
 
-  static class ReadLock implements Lock {
+  public static interface DistributedLock extends Lock {
+    LockType getType();
+  }
+
+  static class ReadLock implements DistributedLock {
 
     QueueLock qlock;
     byte[] userData;
@@ -125,7 +129,8 @@
       this.entry = entry;
     }
 
-    protected LockType lockType() {
+    @Override
+    public LockType getType() {
       return LockType.READ;
     }
 
@@ -154,9 +159,9 @@
     @Override
     public boolean tryLock() {
       if (entry == -1) {
-        entry = qlock.addEntry(new ParsedLock(this.lockType(), this.userData).getLockData());
+        entry = qlock.addEntry(new ParsedLock(this.getType(), this.userData).getLockData());
         log.info("Added lock entry {} userData {} lockType {}", entry,
-            new String(this.userData, UTF_8), lockType());
+            new String(this.userData, UTF_8), getType());
       }
       SortedMap<Long,byte[]> entries = qlock.getEarlierEntries(entry);
       for (Entry<Long,byte[]> entry : entries.entrySet()) {
@@ -169,7 +174,7 @@
         }
       }
       throw new IllegalStateException("Did not find our own lock in the queue: " + this.entry
-          + " userData " + new String(this.userData, UTF_8) + " lockType " + lockType());
+          + " userData " + new String(this.userData, UTF_8) + " lockType " + getType());
     }
 
     @Override
@@ -193,7 +198,7 @@
         return;
       }
       log.debug("Removing lock entry {} userData {} lockType {}", entry,
-          new String(this.userData, UTF_8), lockType());
+          new String(this.userData, UTF_8), getType());
       qlock.removeEntry(entry);
       entry = -1;
     }
@@ -215,22 +220,22 @@
     }
 
     @Override
-    protected LockType lockType() {
+    public LockType getType() {
       return LockType.WRITE;
     }
 
     @Override
     public boolean tryLock() {
       if (entry == -1) {
-        entry = qlock.addEntry(new ParsedLock(this.lockType(), this.userData).getLockData());
+        entry = qlock.addEntry(new ParsedLock(this.getType(), this.userData).getLockData());
         log.info("Added lock entry {} userData {} lockType {}", entry,
-            new String(this.userData, UTF_8), lockType());
+            new String(this.userData, UTF_8), getType());
       }
       SortedMap<Long,byte[]> entries = qlock.getEarlierEntries(entry);
       Iterator<Entry<Long,byte[]>> iterator = entries.entrySet().iterator();
       if (!iterator.hasNext()) {
         throw new IllegalStateException("Did not find our own lock in the queue: " + this.entry
-            + " userData " + new String(this.userData, UTF_8) + " lockType " + lockType());
+            + " userData " + new String(this.userData, UTF_8) + " lockType " + getType());
       }
       return iterator.next().getKey().equals(entry);
     }
@@ -244,7 +249,7 @@
     this.data = Arrays.copyOf(data, data.length);
   }
 
-  public static Lock recoverLock(QueueLock qlock, byte[] data) {
+  public static DistributedLock recoverLock(QueueLock qlock, byte[] data) {
     SortedMap<Long,byte[]> entries = qlock.getEarlierEntries(Long.MAX_VALUE);
     for (Entry<Long,byte[]> entry : entries.entrySet()) {
       ParsedLock parsed = new ParsedLock(entry.getValue());
@@ -261,12 +266,12 @@
   }
 
   @Override
-  public Lock readLock() {
+  public DistributedLock readLock() {
     return new ReadLock(qlock, data);
   }
 
   @Override
-  public Lock writeLock() {
+  public DistributedLock writeLock() {
     return new WriteLock(qlock, data);
   }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/FateLock.java b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/FateLock.java
index fa0e4db..effa070 100644
--- a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/FateLock.java
+++ b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/FateLock.java
@@ -83,8 +83,8 @@
           zoo.putPersistentData(path.toString(), new byte[] {}, NodeExistsPolicy.SKIP);
         }
       }
-    } catch (Exception ex) {
-      throw new RuntimeException(ex);
+    } catch (KeeperException | InterruptedException ex) {
+      throw new IllegalStateException(ex);
     }
   }
 
@@ -112,8 +112,8 @@
           // ignored
         }
       }
-    } catch (Exception ex) {
-      throw new RuntimeException(ex);
+    } catch (KeeperException | InterruptedException ex) {
+      throw new IllegalStateException(ex);
     }
     return result;
   }
@@ -128,8 +128,8 @@
       } catch (NotEmptyException nee) {
         // the path had other lock nodes, no big deal
       }
-    } catch (Exception ex) {
-      throw new RuntimeException(ex);
+    } catch (KeeperException | InterruptedException ex) {
+      throw new IllegalStateException(ex);
     }
   }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooCache.java b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooCache.java
index fe8807c..fc2b6f1 100644
--- a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooCache.java
+++ b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooCache.java
@@ -19,19 +19,22 @@
 package org.apache.accumulo.core.fate.zookeeper;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 
-import java.security.SecureRandom;
 import java.util.Collections;
 import java.util.ConcurrentModificationException;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.LockSupport;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock.ServiceLockPath;
+import org.apache.accumulo.core.lock.ServiceLock;
+import org.apache.accumulo.core.lock.ServiceLock.ServiceLockPath;
+import org.apache.accumulo.core.lock.ServiceLockData;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.KeeperException.Code;
 import org.apache.zookeeper.WatchedEvent;
@@ -62,7 +65,6 @@
   private final HashMap<String,List<String>> childrenCache;
 
   private final ZooReader zReader;
-  private static final SecureRandom random = new SecureRandom();
 
   private volatile boolean closed = false;
 
@@ -270,7 +272,7 @@
         }
         LockSupport.parkNanos(sleepTime);
         if (sleepTime < 10_000) {
-          sleepTime = (int) (sleepTime + sleepTime * random.nextDouble());
+          sleepTime = (int) (sleepTime + sleepTime * RANDOM.get().nextDouble());
         }
       }
     }
@@ -525,13 +527,21 @@
     }
   }
 
-  public byte[] getLockData(ServiceLockPath path) {
+  public Optional<ServiceLockData> getLockData(ServiceLockPath path) {
     List<String> children = ServiceLock.validateAndSort(path, getChildren(path.toString()));
     if (children == null || children.isEmpty()) {
-      return null;
+      return Optional.empty();
     }
     String lockNode = children.get(0);
-    return get(path + "/" + lockNode);
+
+    byte[] lockData = get(path + "/" + lockNode);
+    if (log.isTraceEnabled()) {
+      log.trace("Data from lockNode {} is {}", lockNode, new String(lockData, UTF_8));
+    }
+    if (lockData == null) {
+      lockData = new byte[0];
+    }
+    return ServiceLockData.parse(lockData);
   }
 
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooCacheFactory.java b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooCacheFactory.java
index 39fd5a2..e30c942 100644
--- a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooCacheFactory.java
+++ b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooCacheFactory.java
@@ -27,10 +27,12 @@
 /**
  * A factory for {@link ZooCache} instances.
  * <p>
- * Implementation note: We are using the instances map to track all the instances that have been
- * created, so we can explicitly close them when the last legacy client has gone away. This is part
- * of the "SingletonManager" code, and it is likely that ZooCacheFactory and ZooKeeperInstance can
- * be removed when legacy client code support is no longer required.
+ * Implementation note: We were using the instances map to track all the instances that have been
+ * created, so we could explicitly close them when the SingletonManager detected that the last
+ * legacy client (using Connector/ZooKeeperInstance) has gone away. This class may no longer be
+ * needed, since the legacy client code has been removed, so long as the ZooCache instances it is
+ * tracking are managed as resources within ClientContext or ServerContext, and explicitly closed
+ * when those are closed.
  */
 public class ZooCacheFactory {
 
diff --git a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooReader.java b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooReader.java
index dbd80b2..1df0b54 100644
--- a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooReader.java
+++ b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooReader.java
@@ -19,9 +19,8 @@
 package org.apache.accumulo.core.fate.zookeeper;
 
 import static java.util.Objects.requireNonNull;
-import static java.util.concurrent.TimeUnit.MILLISECONDS;
-import static java.util.concurrent.TimeUnit.MINUTES;
 
+import java.time.Duration;
 import java.util.List;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -42,8 +41,9 @@
   private static final Logger log = LoggerFactory.getLogger(ZooReader.class);
 
   protected static final RetryFactory RETRY_FACTORY =
-      Retry.builder().maxRetries(10).retryAfter(250, MILLISECONDS).incrementBy(250, MILLISECONDS)
-          .maxWait(2, MINUTES).backOffFactor(1.5).logInterval(3, MINUTES).createFactory();
+      Retry.builder().maxRetries(10).retryAfter(Duration.ofMillis(250))
+          .incrementBy(Duration.ofMillis(250)).maxWait(Duration.ofMinutes(2)).backOffFactor(1.5)
+          .logInterval(Duration.ofMinutes(3)).createFactory();
 
   protected final String keepers;
   protected final int timeout;
diff --git a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooSession.java b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooSession.java
index 2f94173..91085f2 100644
--- a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooSession.java
+++ b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooSession.java
@@ -20,10 +20,10 @@
 
 import static java.nio.charset.StandardCharsets.UTF_8;
 import static java.util.concurrent.TimeUnit.NANOSECONDS;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 
 import java.io.IOException;
 import java.net.UnknownHostException;
-import java.security.SecureRandom;
 import java.util.HashMap;
 import java.util.Map;
 
@@ -63,8 +63,6 @@
 
   private static Map<String,ZooSessionInfo> sessions = new HashMap<>();
 
-  private static final SecureRandom random = new SecureRandom();
-
   static {
     SingletonManager.register(new SingletonService() {
 
@@ -157,7 +155,7 @@
       long duration = NANOSECONDS.toMillis(stopTime - startTime);
 
       if (duration > 2L * timeout) {
-        throw new RuntimeException("Failed to connect to zookeeper (" + host
+        throw new IllegalStateException("Failed to connect to zookeeper (" + host
             + ") within 2x zookeeper timeout period " + timeout);
       }
 
@@ -171,7 +169,7 @@
         }
         UtilWaitThread.sleep(sleepTime);
         if (sleepTime < 10000) {
-          sleepTime = sleepTime + (long) (sleepTime * random.nextDouble());
+          sleepTime = sleepTime + (long) (sleepTime * RANDOM.get().nextDouble());
         }
       }
     }
diff --git a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooUtil.java b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooUtil.java
index c968c33..47d906f 100644
--- a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooUtil.java
+++ b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooUtil.java
@@ -27,6 +27,7 @@
 import java.time.ZoneOffset;
 import java.time.format.DateTimeFormatter;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 
 import org.apache.accumulo.core.Constants;
@@ -94,15 +95,18 @@
     }
   }
 
-  public static final List<ACL> PRIVATE;
-  public static final List<ACL> PUBLIC;
+  // Need to use Collections.unmodifiableList() instead of List.of() or List.copyOf(), because
+  // ImmutableCollections.contains() doesn't handle nulls properly (JDK-8265905) and ZooKeeper (as
+  // of 3.8.1) calls acl.contains((Object) null) which throws a NPE when passed an immutable
+  // collection
+  public static final List<ACL> PRIVATE =
+      Collections.unmodifiableList(new ArrayList<>(Ids.CREATOR_ALL_ACL));
 
+  public static final List<ACL> PUBLIC;
   static {
-    PRIVATE = new ArrayList<>();
-    PRIVATE.addAll(Ids.CREATOR_ALL_ACL);
-    PUBLIC = new ArrayList<>();
-    PUBLIC.addAll(PRIVATE);
-    PUBLIC.add(new ACL(Perms.READ, Ids.ANYONE_ID_UNSAFE));
+    var publicTmp = new ArrayList<>(PRIVATE);
+    publicTmp.add(new ACL(Perms.READ, Ids.ANYONE_ID_UNSAFE));
+    PUBLIC = Collections.unmodifiableList(publicTmp);
   }
 
   public static String getRoot(final InstanceId instanceId) {
diff --git a/core/src/main/java/org/apache/accumulo/core/file/BloomFilterLayer.java b/core/src/main/java/org/apache/accumulo/core/file/BloomFilterLayer.java
index 1620f80..fd8d4cb 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/BloomFilterLayer.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/BloomFilterLayer.java
@@ -19,12 +19,12 @@
 package org.apache.accumulo.core.file;
 
 import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.io.PrintStream;
-import java.security.SecureRandom;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -50,11 +50,13 @@
 import org.apache.accumulo.core.file.rfile.RFile;
 import org.apache.accumulo.core.iterators.IteratorEnvironment;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
+import org.apache.accumulo.core.metadata.ReferencedTabletFile;
 import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
 import org.apache.accumulo.core.spi.crypto.NoCryptoServiceFactory;
 import org.apache.accumulo.core.util.threads.ThreadPools;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.util.bloom.Key;
 import org.apache.hadoop.util.hash.Hash;
@@ -67,7 +69,6 @@
  */
 public class BloomFilterLayer {
 
-  private static final SecureRandom random = new SecureRandom();
   private static final Logger LOG = LoggerFactory.getLogger(BloomFilterLayer.class);
   public static final String BLOOM_FILE_NAME = "acu_bloom";
   public static final int HASH_COUNT = 5;
@@ -217,7 +218,7 @@
       final String context = ClassLoaderUtil.tableContext(acuconf);
 
       loadTask = () -> {
-        // no need to load the bloom filter if the map file is closed
+        // no need to load the bloom filter if the data file is closed
         if (closed) {
           return;
         }
@@ -386,13 +387,13 @@
     }
 
     @Override
-    public org.apache.accumulo.core.data.Key getFirstKey() throws IOException {
-      return reader.getFirstKey();
+    public Text getFirstRow() throws IOException {
+      return reader.getFirstRow();
     }
 
     @Override
-    public org.apache.accumulo.core.data.Key getLastKey() throws IOException {
-      return reader.getLastKey();
+    public Text getLastRow() throws IOException {
+      return reader.getLastRow();
     }
 
     @Override
@@ -460,7 +461,7 @@
     HashSet<Integer> valsSet = new HashSet<>();
 
     for (int i = 0; i < 100000; i++) {
-      valsSet.add(random.nextInt(Integer.MAX_VALUE));
+      valsSet.add(RANDOM.get().nextInt(Integer.MAX_VALUE));
     }
 
     ArrayList<Integer> vals = new ArrayList<>(valsSet);
@@ -480,8 +481,8 @@
     String suffix = FileOperations.getNewFileExtension(acuconf);
     String fname = "/tmp/test." + suffix;
     FileSKVWriter bmfw = FileOperations.getInstance().newWriterBuilder()
-        .forFile(fname, fs, conf, NoCryptoServiceFactory.NONE).withTableConfiguration(acuconf)
-        .build();
+        .forFile(new ReferencedTabletFile(new Path(fname)), fs, conf, NoCryptoServiceFactory.NONE)
+        .withTableConfiguration(acuconf).build();
 
     long t1 = System.currentTimeMillis();
 
@@ -503,8 +504,8 @@
 
     t1 = System.currentTimeMillis();
     FileSKVIterator bmfr = FileOperations.getInstance().newReaderBuilder()
-        .forFile(fname, fs, conf, NoCryptoServiceFactory.NONE).withTableConfiguration(acuconf)
-        .build();
+        .forFile(new ReferencedTabletFile(new Path(fname)), fs, conf, NoCryptoServiceFactory.NONE)
+        .withTableConfiguration(acuconf).build();
     t2 = System.currentTimeMillis();
     out.println("Opened " + fname + " in " + (t2 - t1));
 
@@ -512,7 +513,7 @@
 
     int hits = 0;
     for (int i = 0; i < 5000; i++) {
-      int row = random.nextInt(Integer.MAX_VALUE);
+      int row = RANDOM.get().nextInt(Integer.MAX_VALUE);
       String fi = String.format("%010d", row);
       // bmfr.seek(new Range(new Text("r"+fi)));
       org.apache.accumulo.core.data.Key k1 =
diff --git a/core/src/main/java/org/apache/accumulo/core/file/DispatchingFileFactory.java b/core/src/main/java/org/apache/accumulo/core/file/DispatchingFileFactory.java
index fb16b85..f7ba7d2 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/DispatchingFileFactory.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/DispatchingFileFactory.java
@@ -20,25 +20,19 @@
 
 import java.io.IOException;
 
-import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.file.map.MapFileOperations;
 import org.apache.accumulo.core.file.rfile.RFile;
 import org.apache.accumulo.core.file.rfile.RFileOperations;
+import org.apache.accumulo.core.metadata.TabletFile;
 import org.apache.accumulo.core.summary.SummaryWriter;
-import org.apache.hadoop.fs.Path;
 
 class DispatchingFileFactory extends FileOperations {
 
   private FileOperations findFileFactory(FileOptions options) {
-    String file = options.getFilename();
+    TabletFile file = options.getFile();
 
-    Path p = new Path(file);
-    String name = p.getName();
+    String name = file.getPath().getName();
 
-    if (name.startsWith(Constants.MAPFILE_EXTENSION + "_")) {
-      return new MapFileOperations();
-    }
     String[] sp = name.split("\\.");
 
     if (sp.length < 2) {
@@ -47,10 +41,7 @@
 
     String extension = sp[sp.length - 1];
 
-    if (extension.equals(Constants.MAPFILE_EXTENSION)
-        || extension.equals(Constants.MAPFILE_EXTENSION + "_tmp")) {
-      return new MapFileOperations();
-    } else if (extension.equals(RFile.EXTENSION) || extension.equals(RFile.EXTENSION + "_tmp")) {
+    if (extension.equals(RFile.EXTENSION) || extension.equals(RFile.EXTENSION + "_tmp")) {
       return new RFileOperations();
     } else {
       throw new IllegalArgumentException("File type " + extension + " not supported");
diff --git a/core/src/main/java/org/apache/accumulo/core/file/FileOperations.java b/core/src/main/java/org/apache/accumulo/core/file/FileOperations.java
index db82b0d..5182c61 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/FileOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/FileOperations.java
@@ -31,22 +31,24 @@
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.file.blockfile.impl.CacheProvider;
 import org.apache.accumulo.core.file.rfile.RFile;
+import org.apache.accumulo.core.metadata.TabletFile;
+import org.apache.accumulo.core.metadata.UnreferencedTabletFile;
 import org.apache.accumulo.core.spi.crypto.CryptoService;
 import org.apache.accumulo.core.util.ratelimit.RateLimiter;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.FileOutputCommitter;
 
-import com.google.common.cache.Cache;
+import com.github.benmanes.caffeine.cache.Cache;
 
 public abstract class FileOperations {
 
   private static final String HADOOP_JOBHISTORY_LOCATION = "_logs"; // dir related to
                                                                     // hadoop.job.history.user.location
 
-  private static final Set<String> validExtensions =
-      Set.of(Constants.MAPFILE_EXTENSION, RFile.EXTENSION);
+  private static final Set<String> validExtensions = Set.of(RFile.EXTENSION);
 
   // Sometimes we want to know what files accumulo bulk processing creates
   private static final Set<String> bulkWorkingFiles =
@@ -163,10 +165,10 @@
     return new ReaderBuilder();
   }
 
-  public static class FileOptions {
+  protected static class FileOptions {
     // objects used by all
     public final AccumuloConfiguration tableConfiguration;
-    public final String filename;
+    public final TabletFile file;
     public final FileSystem fs;
     public final Configuration fsConf;
     public final RateLimiter rateLimiter;
@@ -185,13 +187,13 @@
     public final boolean inclusive;
     public final boolean dropCacheBehind;
 
-    public FileOptions(AccumuloConfiguration tableConfiguration, String filename, FileSystem fs,
+    protected FileOptions(AccumuloConfiguration tableConfiguration, TabletFile file, FileSystem fs,
         Configuration fsConf, RateLimiter rateLimiter, String compression,
         FSDataOutputStream outputStream, boolean enableAccumuloStart, CacheProvider cacheProvider,
         Cache<String,Long> fileLenCache, boolean seekToBeginning, CryptoService cryptoService,
         Range range, Set<ByteSequence> columnFamilies, boolean inclusive, boolean dropCacheBehind) {
       this.tableConfiguration = tableConfiguration;
-      this.filename = filename;
+      this.file = Objects.requireNonNull(file);
       this.fs = fs;
       this.fsConf = fsConf;
       this.rateLimiter = rateLimiter;
@@ -212,8 +214,8 @@
       return tableConfiguration;
     }
 
-    public String getFilename() {
-      return filename;
+    public TabletFile getFile() {
+      return file;
     }
 
     public FileSystem getFileSystem() {
@@ -274,7 +276,7 @@
    */
   public static class FileHelper {
     private AccumuloConfiguration tableConfiguration;
-    private String filename;
+    private TabletFile file;
     private FileSystem fs;
     private Configuration fsConf;
     private RateLimiter rateLimiter;
@@ -291,8 +293,8 @@
       return this;
     }
 
-    protected FileHelper filename(String filename) {
-      this.filename = Objects.requireNonNull(filename);
+    protected FileHelper file(TabletFile file) {
+      this.file = Objects.requireNonNull(file);
       return this;
     }
 
@@ -318,28 +320,27 @@
 
     protected FileOptions toWriterBuilderOptions(String compression,
         FSDataOutputStream outputStream, boolean startEnabled) {
-      return new FileOptions(tableConfiguration, filename, fs, fsConf, rateLimiter, compression,
+      return new FileOptions(tableConfiguration, file, fs, fsConf, rateLimiter, compression,
           outputStream, startEnabled, NULL_PROVIDER, null, false, cryptoService, null, null, true,
           dropCacheBehind);
     }
 
     protected FileOptions toReaderBuilderOptions(CacheProvider cacheProvider,
         Cache<String,Long> fileLenCache, boolean seekToBeginning) {
-      return new FileOptions(tableConfiguration, filename, fs, fsConf, rateLimiter, null, null,
-          false, cacheProvider == null ? NULL_PROVIDER : cacheProvider, fileLenCache,
-          seekToBeginning, cryptoService, null, null, true, dropCacheBehind);
+      return new FileOptions(tableConfiguration, file, fs, fsConf, rateLimiter, null, null, false,
+          cacheProvider == null ? NULL_PROVIDER : cacheProvider, fileLenCache, seekToBeginning,
+          cryptoService, null, null, true, dropCacheBehind);
     }
 
     protected FileOptions toIndexReaderBuilderOptions(Cache<String,Long> fileLenCache) {
-      return new FileOptions(tableConfiguration, filename, fs, fsConf, rateLimiter, null, null,
-          false, NULL_PROVIDER, fileLenCache, false, cryptoService, null, null, true,
-          dropCacheBehind);
+      return new FileOptions(tableConfiguration, file, fs, fsConf, rateLimiter, null, null, false,
+          NULL_PROVIDER, fileLenCache, false, cryptoService, null, null, true, dropCacheBehind);
     }
 
     protected FileOptions toScanReaderBuilderOptions(Range range, Set<ByteSequence> columnFamilies,
         boolean inclusive) {
-      return new FileOptions(tableConfiguration, filename, fs, fsConf, rateLimiter, null, null,
-          false, NULL_PROVIDER, null, false, cryptoService, range, columnFamilies, inclusive,
+      return new FileOptions(tableConfiguration, file, fs, fsConf, rateLimiter, null, null, false,
+          NULL_PROVIDER, null, false, cryptoService, range, columnFamilies, inclusive,
           dropCacheBehind);
     }
 
@@ -357,15 +358,17 @@
     private boolean enableAccumuloStart = true;
 
     public WriterTableConfiguration forOutputStream(String extension,
-        FSDataOutputStream outputStream, Configuration fsConf, CryptoService cs) {
+        FSDataOutputStream outputStream, Configuration fsConf, CryptoService cs)
+        throws IOException {
       this.outputStream = outputStream;
-      filename("foo" + extension).fsConf(fsConf).cryptoService(cs);
+      file(UnreferencedTabletFile.of(fsConf, new Path("foo/foo" + extension))).fsConf(fsConf)
+          .cryptoService(cs);
       return this;
     }
 
-    public WriterTableConfiguration forFile(String filename, FileSystem fs, Configuration fsConf,
+    public WriterTableConfiguration forFile(TabletFile file, FileSystem fs, Configuration fsConf,
         CryptoService cs) {
-      filename(filename).fs(fs).fsConf(fsConf).cryptoService(cs);
+      file(file).fs(fs).fsConf(fsConf).cryptoService(cs);
       return this;
     }
 
@@ -412,9 +415,9 @@
     private Cache<String,Long> fileLenCache;
     private boolean seekToBeginning = false;
 
-    public ReaderTableConfiguration forFile(String filename, FileSystem fs, Configuration fsConf,
+    public ReaderTableConfiguration forFile(TabletFile file, FileSystem fs, Configuration fsConf,
         CryptoService cs) {
-      filename(filename).fs(fs).fsConf(fsConf).cryptoService(cs);
+      file(file).fs(fs).fsConf(fsConf).cryptoService(cs);
       return this;
     }
 
@@ -480,9 +483,9 @@
 
     private Cache<String,Long> fileLenCache = null;
 
-    public IndexReaderTableConfiguration forFile(String filename, FileSystem fs,
+    public IndexReaderTableConfiguration forFile(TabletFile file, FileSystem fs,
         Configuration fsConf, CryptoService cs) {
-      filename(filename).fs(fs).fsConf(fsConf).cryptoService(cs);
+      file(file).fs(fs).fsConf(fsConf).cryptoService(cs);
       return this;
     }
 
@@ -512,9 +515,9 @@
     private Set<ByteSequence> columnFamilies;
     private boolean inclusive;
 
-    public ScanReaderTableConfiguration forFile(String filename, FileSystem fs,
+    public ScanReaderTableConfiguration forFile(TabletFile file, FileSystem fs,
         Configuration fsConf, CryptoService cs) {
-      filename(filename).fs(fs).fsConf(fsConf).cryptoService(cs);
+      file(file).fs(fs).fsConf(fsConf).cryptoService(cs);
       return this;
     }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/file/FilePrefix.java b/core/src/main/java/org/apache/accumulo/core/file/FilePrefix.java
new file mode 100644
index 0000000..febec86
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/file/FilePrefix.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.file;
+
+import java.util.stream.Stream;
+
+public enum FilePrefix {
+
+  BULK_IMPORT("I"), MINOR_COMPACTION("F"), MAJOR_COMPACTION("C"), MAJOR_COMPACTION_ALL_FILES("A");
+
+  String prefix;
+
+  FilePrefix(String prefix) {
+    this.prefix = prefix;
+  }
+
+  public static FilePrefix fromPrefix(String prefix) {
+    return Stream.of(FilePrefix.values()).filter(p -> p.prefix.equals(prefix)).findAny()
+        .orElseThrow(() -> new IllegalArgumentException("Unknown prefix type: " + prefix));
+  }
+
+  public String toPrefix() {
+    return this.prefix;
+  }
+
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/file/FileSKVIterator.java b/core/src/main/java/org/apache/accumulo/core/file/FileSKVIterator.java
index 23da429..b55f1ba 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/FileSKVIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/FileSKVIterator.java
@@ -21,16 +21,16 @@
 import java.io.DataInputStream;
 import java.io.IOException;
 
-import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.file.blockfile.impl.CacheProvider;
 import org.apache.accumulo.core.iteratorsImpl.system.InterruptibleIterator;
 import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
+import org.apache.hadoop.io.Text;
 
 public interface FileSKVIterator extends InterruptibleIterator, AutoCloseable {
-  Key getFirstKey() throws IOException;
+  Text getFirstRow() throws IOException;
 
-  Key getLastKey() throws IOException;
+  Text getLastRow() throws IOException;
 
   DataInputStream getMetaStore(String name) throws IOException, NoSuchMetaStoreException;
 
diff --git a/core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/impl/BlockCacheManagerFactory.java b/core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/impl/BlockCacheManagerFactory.java
index b72979d..655b7b3 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/impl/BlockCacheManagerFactory.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/impl/BlockCacheManagerFactory.java
@@ -31,14 +31,14 @@
 
   /**
    * Get the BlockCacheFactory specified by the property 'tserver.cache.factory.class' using the
-   * AccumuloVFSClassLoader
+   * System class loader
    *
    * @param conf accumulo configuration
    * @return block cache manager instance
-   * @throws Exception error loading block cache manager implementation class
+   * @throws ReflectiveOperationException error loading block cache manager implementation class
    */
   public static synchronized BlockCacheManager getInstance(AccumuloConfiguration conf)
-      throws Exception {
+      throws ReflectiveOperationException {
     String impl = conf.get(Property.TSERV_CACHE_MANAGER_IMPL);
     Class<? extends BlockCacheManager> clazz =
         ClassLoaderUtil.loadClass(impl, BlockCacheManager.class);
@@ -51,10 +51,10 @@
    *
    * @param conf accumulo configuration
    * @return block cache manager instance
-   * @throws Exception error loading block cache manager implementation class
+   * @throws ReflectiveOperationException error loading block cache manager implementation class
    */
   public static synchronized BlockCacheManager getClientInstance(AccumuloConfiguration conf)
-      throws Exception {
+      throws ReflectiveOperationException {
     String impl = conf.get(Property.TSERV_CACHE_MANAGER_IMPL);
     Class<? extends BlockCacheManager> clazz =
         Class.forName(impl).asSubclass(BlockCacheManager.class);
diff --git a/core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/lru/LruBlockCache.java b/core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/lru/LruBlockCache.java
index dc85350..7a85da7 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/lru/LruBlockCache.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/lru/LruBlockCache.java
@@ -154,7 +154,7 @@
         try {
           Thread.sleep(10);
         } catch (InterruptedException ex) {
-          throw new RuntimeException(ex);
+          throw new IllegalStateException(ex);
         }
       }
     } else {
@@ -633,6 +633,11 @@
       return accessCount.get();
     }
 
+    @Override
+    public long evictionCount() {
+      return getEvictedCount();
+    }
+
     public long getMissCount() {
       return missCount.get();
     }
diff --git a/core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/tinylfu/TinyLfuBlockCache.java b/core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/tinylfu/TinyLfuBlockCache.java
index 78206c7..7ee9dad 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/tinylfu/TinyLfuBlockCache.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/tinylfu/TinyLfuBlockCache.java
@@ -115,6 +115,11 @@
       public long requestCount() {
         return stats.requestCount();
       }
+
+      @Override
+      public long evictionCount() {
+        return stats.evictionCount();
+      }
     };
   }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/file/blockfile/impl/CachableBlockFile.java b/core/src/main/java/org/apache/accumulo/core/file/blockfile/impl/CachableBlockFile.java
index b54b8ab..0beb67f 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/blockfile/impl/CachableBlockFile.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/blockfile/impl/CachableBlockFile.java
@@ -26,7 +26,6 @@
 import java.util.Collections;
 import java.util.Map;
 import java.util.Objects;
-import java.util.concurrent.ExecutionException;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.function.Supplier;
 
@@ -48,7 +47,7 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.cache.Cache;
+import com.github.benmanes.caffeine.cache.Cache;
 
 /**
  * This is a wrapper class for BCFile that includes a cache for independent caches for datablocks
@@ -170,8 +169,14 @@
 
     private long getCachedFileLen() throws IOException {
       try {
-        return fileLenCache.get(cacheId, lengthSupplier::get);
-      } catch (ExecutionException e) {
+        return fileLenCache.get(cacheId, k -> {
+          try {
+            return lengthSupplier.get();
+          } catch (IOException e) {
+            throw new UncheckedIOException(e);
+          }
+        });
+      } catch (UncheckedIOException e) {
         throw new IOException("Failed to get " + cacheId + " len from cache ", e);
       }
     }
diff --git a/core/src/main/java/org/apache/accumulo/core/file/blockfile/impl/SeekableByteArrayInputStream.java b/core/src/main/java/org/apache/accumulo/core/file/blockfile/impl/SeekableByteArrayInputStream.java
index c231e88..9aaa67b 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/blockfile/impl/SeekableByteArrayInputStream.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/blockfile/impl/SeekableByteArrayInputStream.java
@@ -22,8 +22,9 @@
 
 import java.io.IOException;
 import java.io.InputStream;
-
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.BiFunction;
+import java.util.function.IntBinaryOperator;
 
 /**
  * This class is like byte array input stream with two differences. It supports seeking and avoids
@@ -31,24 +32,16 @@
  */
 public class SeekableByteArrayInputStream extends InputStream {
 
-  // making this volatile for the following case
-  // * thread 1 creates and initializes byte array
-  // * thread 2 reads from bye array
-  // spotbugs complains about this because thread2 may not see any changes to the byte array after
-  // thread 1 set the volatile,
-  // however the expectation is that the byte array is static. In the case of it being static,
-  // volatile ensures that
-  // thread 2 sees all of thread 1 changes before setting the volatile.
-  @SuppressFBWarnings(value = "VO_VOLATILE_REFERENCE_TO_ARRAY",
-      justification = "see explanation above")
-  private volatile byte[] buffer;
-  private int cur;
-  private int max;
+  private final byte[] buffer;
+  private final AtomicInteger cur = new AtomicInteger(0);
+  private final int max;
 
   @Override
   public int read() {
-    if (cur < max) {
-      return buffer[cur++] & 0xff;
+    // advance the pointer by 1 if we haven't reached the end
+    final int currentValue = cur.getAndAccumulate(1, (v, x) -> v < max ? v + x : v);
+    if (currentValue < max) {
+      return buffer[currentValue] & 0xff;
     } else {
       return -1;
     }
@@ -68,7 +61,20 @@
       return 0;
     }
 
-    int avail = max - cur;
+    // compute how much to read, based on what's left available
+    IntBinaryOperator add = (cur1, length1) -> {
+      final int available = max - cur1;
+      if (available <= 0) {
+        return cur1;
+      } else if (length1 > available) {
+        length1 = available;
+      }
+      return cur1 + length1;
+    };
+
+    final int currentValue = cur.getAndAccumulate(length, add);
+
+    final int avail = max - currentValue;
 
     if (avail <= 0) {
       return -1;
@@ -78,29 +84,29 @@
       length = avail;
     }
 
-    System.arraycopy(buffer, cur, b, offset, length);
-    cur += length;
+    System.arraycopy(buffer, currentValue, b, offset, length);
     return length;
   }
 
   @Override
   public long skip(long requestedSkip) {
-    int actualSkip = max - cur;
-    if (requestedSkip < actualSkip) {
-      if (requestedSkip < 0) {
-        actualSkip = 0;
-      } else {
-        actualSkip = (int) requestedSkip;
-      }
-    }
 
-    cur += actualSkip;
-    return actualSkip;
+    // actual skip is at least 0, but no more than what's available
+    BiFunction<Integer,Integer,Integer> skipValue =
+        (current, skip) -> Math.max(0, Math.min(max - current, skip));
+
+    // compute how much to advance, based on actual amount skipped
+    IntBinaryOperator add = (cur1, skip) -> cur1 + skipValue.apply(cur1, skip);
+
+    // advance the pointer and return the actual amount skipped
+    int currentValue = cur.getAndAccumulate((int) requestedSkip, add);
+
+    return skipValue.apply(currentValue, (int) requestedSkip);
   }
 
   @Override
   public int available() {
-    return max - cur;
+    return max - cur.get();
   }
 
   @Override
@@ -124,14 +130,12 @@
   public SeekableByteArrayInputStream(byte[] buf) {
     requireNonNull(buf, "bug argument was null");
     this.buffer = buf;
-    this.cur = 0;
     this.max = buf.length;
   }
 
   public SeekableByteArrayInputStream(byte[] buf, int maxOffset) {
     requireNonNull(buf, "bug argument was null");
     this.buffer = buf;
-    this.cur = 0;
     this.max = maxOffset;
   }
 
@@ -139,11 +143,11 @@
     if (position < 0 || position >= max) {
       throw new IllegalArgumentException("position = " + position + " maxOffset = " + max);
     }
-    this.cur = position;
+    this.cur.set(position);
   }
 
   public int getPosition() {
-    return this.cur;
+    return this.cur.get();
   }
 
   byte[] getBuffer() {
diff --git a/core/src/main/java/org/apache/accumulo/core/file/map/MapFileOperations.java b/core/src/main/java/org/apache/accumulo/core/file/map/MapFileOperations.java
deleted file mode 100644
index 6f4c2e2..0000000
--- a/core/src/main/java/org/apache/accumulo/core/file/map/MapFileOperations.java
+++ /dev/null
@@ -1,192 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.file.map;
-
-import java.io.DataInputStream;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Map;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import org.apache.accumulo.core.data.ByteSequence;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.dataImpl.KeyExtent;
-import org.apache.accumulo.core.file.FileOperations;
-import org.apache.accumulo.core.file.FileSKVIterator;
-import org.apache.accumulo.core.file.FileSKVWriter;
-import org.apache.accumulo.core.file.blockfile.impl.CacheProvider;
-import org.apache.accumulo.core.iterators.IteratorEnvironment;
-import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
-import org.apache.accumulo.core.iteratorsImpl.system.MapFileIterator;
-import org.apache.accumulo.core.iteratorsImpl.system.SequenceFileIterator;
-import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.MapFile;
-
-public class MapFileOperations extends FileOperations {
-
-  public static class RangeIterator implements FileSKVIterator {
-
-    SortedKeyValueIterator<Key,Value> reader;
-    private Range range;
-    private boolean hasTop;
-
-    public RangeIterator(SortedKeyValueIterator<Key,Value> reader) {
-      this.reader = reader;
-    }
-
-    @Override
-    public void close() throws IOException {
-      ((FileSKVIterator) reader).close();
-    }
-
-    @Override
-    public Key getFirstKey() throws IOException {
-      return ((FileSKVIterator) reader).getFirstKey();
-    }
-
-    @Override
-    public Key getLastKey() throws IOException {
-      return ((FileSKVIterator) reader).getLastKey();
-    }
-
-    @Override
-    public DataInputStream getMetaStore(String name) throws IOException {
-      return ((FileSKVIterator) reader).getMetaStore(name);
-    }
-
-    @Override
-    public long estimateOverlappingEntries(KeyExtent extent) throws IOException {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
-      return new RangeIterator(reader.deepCopy(env));
-    }
-
-    @Override
-    public Key getTopKey() {
-      if (!hasTop) {
-        throw new IllegalStateException();
-      }
-      return reader.getTopKey();
-    }
-
-    @Override
-    public Value getTopValue() {
-      if (!hasTop) {
-        throw new IllegalStateException();
-      }
-      return reader.getTopValue();
-    }
-
-    @Override
-    public boolean hasTop() {
-      return hasTop;
-    }
-
-    @Override
-    public void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> options,
-        IteratorEnvironment env) {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public void next() throws IOException {
-      if (!hasTop) {
-        throw new IllegalStateException();
-      }
-      reader.next();
-      hasTop = reader.hasTop() && !range.afterEndKey(reader.getTopKey());
-    }
-
-    @Override
-    public void seek(Range range, Collection<ByteSequence> columnFamilies, boolean inclusive)
-        throws IOException {
-      reader.seek(range, columnFamilies, inclusive);
-      this.range = range;
-
-      hasTop = reader.hasTop() && !range.afterEndKey(reader.getTopKey());
-
-      while (hasTop() && range.beforeStartKey(getTopKey())) {
-        next();
-      }
-    }
-
-    @Override
-    public void closeDeepCopies() throws IOException {
-      ((FileSKVIterator) reader).closeDeepCopies();
-    }
-
-    @Override
-    public void setInterruptFlag(AtomicBoolean flag) {
-      ((FileSKVIterator) reader).setInterruptFlag(flag);
-    }
-
-    @Override
-    public FileSKVIterator getSample(SamplerConfigurationImpl sampleConfig) {
-      return ((FileSKVIterator) reader).getSample(sampleConfig);
-    }
-
-    @Override
-    public void setCacheProvider(CacheProvider cacheProvider) {}
-  }
-
-  @Override
-  protected FileSKVIterator openReader(FileOptions options) throws IOException {
-    FileSKVIterator iter = new RangeIterator(new MapFileIterator(options.getFileSystem(),
-        options.getFilename(), options.getConfiguration()));
-    if (options.isSeekToBeginning()) {
-      iter.seek(new Range(new Key(), null), new ArrayList<>(), false);
-    }
-    return iter;
-  }
-
-  @Override
-  protected FileSKVWriter openWriter(FileOptions options) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  protected FileSKVIterator openIndex(FileOptions options) throws IOException {
-    return new SequenceFileIterator(MapFileUtil.openIndex(options.getConfiguration(),
-        options.getFileSystem(), new Path(options.getFilename())), false);
-  }
-
-  @Override
-  protected long getFileSize(FileOptions options) throws IOException {
-    return options.getFileSystem()
-        .getFileStatus(new Path(options.getFilename() + "/" + MapFile.DATA_FILE_NAME)).getLen();
-  }
-
-  @Override
-  protected FileSKVIterator openScanReader(FileOptions options) throws IOException {
-    MapFileIterator mfIter = new MapFileIterator(options.getFileSystem(), options.getFilename(),
-        options.getConfiguration());
-
-    FileSKVIterator iter = new RangeIterator(mfIter);
-    iter.seek(options.getRange(), options.getColumnFamilies(), options.isRangeInclusive());
-
-    return iter;
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/file/map/MapFileUtil.java b/core/src/main/java/org/apache/accumulo/core/file/map/MapFileUtil.java
deleted file mode 100644
index 7fd7276..0000000
--- a/core/src/main/java/org/apache/accumulo/core/file/map/MapFileUtil.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.file.map;
-
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.MapFile;
-import org.apache.hadoop.io.SequenceFile;
-
-public class MapFileUtil {
-  public static MapFile.Reader openMapFile(FileSystem fs, String dirName, Configuration conf)
-      throws IOException {
-    MapFile.Reader mfr = null;
-    try {
-      mfr = new MapFile.Reader(fs.makeQualified(new Path(dirName)), conf);
-      return mfr;
-    } catch (IOException e) {
-      throw e;
-    }
-  }
-
-  public static SequenceFile.Reader openIndex(Configuration conf, FileSystem fs, Path mapFile)
-      throws IOException {
-    Path indexPath = new Path(mapFile, MapFile.INDEX_FILE_NAME);
-    SequenceFile.Reader index = null;
-    try {
-      index = new SequenceFile.Reader(conf, SequenceFile.Reader.file(fs.makeQualified(indexPath)));
-      return index;
-    } catch (IOException e) {
-      throw e;
-    }
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/file/rfile/CreateEmpty.java b/core/src/main/java/org/apache/accumulo/core/file/rfile/CreateEmpty.java
deleted file mode 100644
index f7d7eb9..0000000
--- a/core/src/main/java/org/apache/accumulo/core/file/rfile/CreateEmpty.java
+++ /dev/null
@@ -1,112 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.file.rfile;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.accumulo.core.cli.Help;
-import org.apache.accumulo.core.conf.DefaultConfiguration;
-import org.apache.accumulo.core.file.FileSKVWriter;
-import org.apache.accumulo.core.file.rfile.bcfile.Compression;
-import org.apache.accumulo.core.spi.crypto.NoCryptoServiceFactory;
-import org.apache.accumulo.core.spi.file.rfile.compression.NoCompression;
-import org.apache.accumulo.start.spi.KeywordExecutable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.beust.jcommander.IParameterValidator;
-import com.beust.jcommander.Parameter;
-import com.beust.jcommander.ParameterException;
-import com.google.auto.service.AutoService;
-
-/**
- * Create an empty RFile for use in recovering from data loss where Accumulo still refers internally
- * to a path.
- */
-@AutoService(KeywordExecutable.class)
-public class CreateEmpty implements KeywordExecutable {
-  private static final Logger log = LoggerFactory.getLogger(CreateEmpty.class);
-
-  public static class NamedLikeRFile implements IParameterValidator {
-    @Override
-    public void validate(String name, String value) throws ParameterException {
-      if (!value.endsWith(".rf")) {
-        throw new ParameterException("File must end with .rf and '" + value + "' does not.");
-      }
-    }
-  }
-
-  public static class IsSupportedCompressionAlgorithm implements IParameterValidator {
-    @Override
-    public void validate(String name, String value) throws ParameterException {
-      List<String> algorithms = Compression.getSupportedAlgorithms();
-      if (!algorithms.contains(value)) {
-        throw new ParameterException("Compression codec must be one of " + algorithms);
-      }
-    }
-  }
-
-  static class Opts extends Help {
-    @Parameter(names = {"-c", "--codec"}, description = "the compression codec to use.",
-        validateWith = IsSupportedCompressionAlgorithm.class)
-    String codec = new NoCompression().getName();
-    @Parameter(
-        description = " <path> { <path> ... } Each path given is a URL."
-            + " Relative paths are resolved according to the default filesystem defined in"
-            + " your Hadoop configuration, which is usually an HDFS instance.",
-        required = true, validateWith = NamedLikeRFile.class)
-    List<String> files = new ArrayList<>();
-  }
-
-  public static void main(String[] args) throws Exception {
-    new CreateEmpty().execute(args);
-  }
-
-  @Override
-  public String keyword() {
-    return "create-empty";
-  }
-
-  @Override
-  public String description() {
-    return "Creates an empty rfile";
-  }
-
-  @Override
-  public void execute(String[] args) throws Exception {
-    Configuration conf = new Configuration();
-
-    Opts opts = new Opts();
-    opts.parseArgs("accumulo create-empty", args);
-
-    for (String arg : opts.files) {
-      Path path = new Path(arg);
-      log.info("Writing to file '{}'", path);
-      FileSKVWriter writer = new RFileOperations().newWriterBuilder()
-          .forFile(arg, path.getFileSystem(conf), conf, NoCryptoServiceFactory.NONE)
-          .withTableConfiguration(DefaultConfiguration.getInstance()).withCompression(opts.codec)
-          .build();
-      writer.close();
-    }
-  }
-
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/file/rfile/GenerateSplits.java b/core/src/main/java/org/apache/accumulo/core/file/rfile/GenerateSplits.java
index 865210a..7ee151f 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/rfile/GenerateSplits.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/rfile/GenerateSplits.java
@@ -47,6 +47,7 @@
 import org.apache.accumulo.core.file.FileSKVIterator;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 import org.apache.accumulo.core.iteratorsImpl.system.MultiIterator;
+import org.apache.accumulo.core.metadata.UnreferencedTabletFile;
 import org.apache.accumulo.core.spi.crypto.CryptoEnvironment;
 import org.apache.accumulo.core.spi.crypto.CryptoService;
 import org.apache.accumulo.core.util.TextUtil;
@@ -138,18 +139,18 @@
     long splitSize = opts.splitSize;
 
     FileSystem fs = FileSystem.get(hadoopConf);
-    List<Path> filePaths = new ArrayList<>();
+    List<UnreferencedTabletFile> files = new ArrayList<>();
     for (String file : opts.files) {
       Path path = new Path(file);
       fs = PrintInfo.resolveFS(log, hadoopConf, path);
       // get all the files in the directory
-      filePaths.addAll(getFiles(fs, path));
+      files.addAll(getFiles(fs, path));
     }
 
-    if (filePaths.isEmpty()) {
+    if (files.isEmpty()) {
       throw new IllegalArgumentException("No files were found in " + opts.files);
     } else {
-      log.trace("Found the following files: {}", filePaths);
+      log.trace("Found the following files: {}", files);
     }
 
     if (!encode) {
@@ -168,18 +169,17 @@
 
     // if no size specified look at indexed keys first
     if (opts.splitSize == 0) {
-      splits = getIndexKeys(siteConf, hadoopConf, fs, filePaths, requestedNumSplits, encode,
-          cryptoService);
+      splits =
+          getIndexKeys(siteConf, hadoopConf, fs, files, requestedNumSplits, encode, cryptoService);
       // if there weren't enough splits indexed, try again with size = 0
       if (splits.size() < requestedNumSplits) {
         log.info("Only found {} indexed keys but need {}. Doing a full scan on files {}",
-            splits.size(), requestedNumSplits, filePaths);
-        splits = getSplitsFromFullScan(siteConf, hadoopConf, filePaths, fs, requestedNumSplits,
-            encode, cryptoService);
+            splits.size(), requestedNumSplits, files);
+        splits = getSplitsFromFullScan(siteConf, hadoopConf, files, fs, requestedNumSplits, encode,
+            cryptoService);
       }
     } else {
-      splits =
-          getSplitsBySize(siteConf, hadoopConf, filePaths, fs, splitSize, encode, cryptoService);
+      splits = getSplitsBySize(siteConf, hadoopConf, files, fs, splitSize, encode, cryptoService);
     }
 
     TreeSet<String> desiredSplits;
@@ -205,20 +205,20 @@
     }
   }
 
-  private List<Path> getFiles(FileSystem fs, Path path) throws IOException {
-    List<Path> filePaths = new ArrayList<>();
+  private List<UnreferencedTabletFile> getFiles(FileSystem fs, Path path) throws IOException {
+    List<UnreferencedTabletFile> files = new ArrayList<>();
     if (fs.getFileStatus(path).isDirectory()) {
       var iter = fs.listFiles(path, true);
       while (iter.hasNext()) {
-        filePaths.addAll(getFiles(fs, iter.next().getPath()));
+        files.addAll(getFiles(fs, iter.next().getPath()));
       }
     } else {
       if (!path.toString().endsWith(".rf")) {
         throw new IllegalArgumentException("Provided file (" + path + ") does not end with '.rf'");
       }
-      filePaths.add(path);
+      files.add(UnreferencedTabletFile.of(fs, path));
     }
-    return filePaths;
+    return files;
   }
 
   private Text[] getQuantiles(SortedKeyValueIterator<Key,Value> iterator, int numSplits)
@@ -296,16 +296,15 @@
    * Scan the files for indexed keys first since it is more efficient than a full file scan.
    */
   private TreeSet<String> getIndexKeys(AccumuloConfiguration accumuloConf, Configuration hadoopConf,
-      FileSystem fs, List<Path> files, int requestedNumSplits, boolean base64encode,
-      CryptoService cs) throws IOException {
+      FileSystem fs, List<UnreferencedTabletFile> files, int requestedNumSplits,
+      boolean base64encode, CryptoService cs) throws IOException {
     Text[] splitArray;
     List<SortedKeyValueIterator<Key,Value>> readers = new ArrayList<>(files.size());
     List<FileSKVIterator> fileReaders = new ArrayList<>(files.size());
     try {
-      for (Path file : files) {
+      for (UnreferencedTabletFile file : files) {
         FileSKVIterator reader = FileOperations.getInstance().newIndexReaderBuilder()
-            .forFile(file.toString(), fs, hadoopConf, cs).withTableConfiguration(accumuloConf)
-            .build();
+            .forFile(file, fs, hadoopConf, cs).withTableConfiguration(accumuloConf).build();
         readers.add(reader);
         fileReaders.add(reader);
       }
@@ -323,7 +322,7 @@
   }
 
   private TreeSet<String> getSplitsFromFullScan(SiteConfiguration accumuloConf,
-      Configuration hadoopConf, List<Path> files, FileSystem fs, int numSplits,
+      Configuration hadoopConf, List<UnreferencedTabletFile> files, FileSystem fs, int numSplits,
       boolean base64encode, CryptoService cs) throws IOException {
     Text[] splitArray;
     List<FileSKVIterator> fileReaders = new ArrayList<>(files.size());
@@ -331,9 +330,9 @@
     SortedKeyValueIterator<Key,Value> iterator;
 
     try {
-      for (Path file : files) {
+      for (UnreferencedTabletFile file : files) {
         FileSKVIterator reader = FileOperations.getInstance().newScanReaderBuilder()
-            .forFile(file.toString(), fs, hadoopConf, cs).withTableConfiguration(accumuloConf)
+            .forFile(file, fs, hadoopConf, cs).withTableConfiguration(accumuloConf)
             .overRange(new Range(), Set.of(), false).build();
         readers.add(reader);
         fileReaders.add(reader);
@@ -356,7 +355,7 @@
    * Get number of splits based on requested size of split.
    */
   private TreeSet<String> getSplitsBySize(AccumuloConfiguration accumuloConf,
-      Configuration hadoopConf, List<Path> files, FileSystem fs, long splitSize,
+      Configuration hadoopConf, List<UnreferencedTabletFile> files, FileSystem fs, long splitSize,
       boolean base64encode, CryptoService cs) throws IOException {
     long currentSplitSize = 0;
     long totalSize = 0;
@@ -365,9 +364,9 @@
     List<SortedKeyValueIterator<Key,Value>> readers = new ArrayList<>(files.size());
     SortedKeyValueIterator<Key,Value> iterator;
     try {
-      for (Path file : files) {
+      for (UnreferencedTabletFile file : files) {
         FileSKVIterator reader = FileOperations.getInstance().newScanReaderBuilder()
-            .forFile(file.toString(), fs, hadoopConf, cs).withTableConfiguration(accumuloConf)
+            .forFile(file, fs, hadoopConf, cs).withTableConfiguration(accumuloConf)
             .overRange(new Range(), Set.of(), false).build();
         readers.add(reader);
         fileReaders.add(reader);
diff --git a/core/src/main/java/org/apache/accumulo/core/file/rfile/MultiIndexIterator.java b/core/src/main/java/org/apache/accumulo/core/file/rfile/MultiIndexIterator.java
index d9fc362..4fb3600 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/rfile/MultiIndexIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/rfile/MultiIndexIterator.java
@@ -38,6 +38,7 @@
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 import org.apache.accumulo.core.iteratorsImpl.system.HeapIterator;
 import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
+import org.apache.hadoop.io.Text;
 
 class MultiIndexIterator extends HeapIterator implements FileSKVIterator {
 
@@ -81,12 +82,12 @@
   }
 
   @Override
-  public Key getFirstKey() throws IOException {
+  public Text getFirstRow() throws IOException {
     throw new UnsupportedOperationException();
   }
 
   @Override
-  public Key getLastKey() throws IOException {
+  public Text getLastRow() throws IOException {
     throw new UnsupportedOperationException();
   }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/file/rfile/MultiLevelIndex.java b/core/src/main/java/org/apache/accumulo/core/file/rfile/MultiLevelIndex.java
index 770b698..0760e0f 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/rfile/MultiLevelIndex.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/rfile/MultiLevelIndex.java
@@ -28,6 +28,7 @@
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.io.PrintStream;
+import java.io.UncheckedIOException;
 import java.util.AbstractList;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -190,7 +191,7 @@
         sbais.seek(indexOffset + offset);
         return newValue();
       } catch (IOException ioe) {
-        throw new RuntimeException(ioe);
+        throw new UncheckedIOException(ioe);
       }
     }
 
@@ -399,7 +400,7 @@
         offsetsArray = offsets;
         newFormat = false;
       } else {
-        throw new RuntimeException("Unexpected version " + version);
+        throw new IllegalStateException("Unexpected version " + version);
       }
 
     }
@@ -705,7 +706,7 @@
         try {
           return node.getPreviousNode();
         } catch (IOException e) {
-          throw new RuntimeException(e);
+          throw new UncheckedIOException(e);
         }
       }
 
@@ -713,7 +714,7 @@
         try {
           return node.getNextNode();
         } catch (IOException e) {
-          throw new RuntimeException(e);
+          throw new UncheckedIOException(e);
         }
       }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/file/rfile/RFile.java b/core/src/main/java/org/apache/accumulo/core/file/rfile/RFile.java
index 059eac9..00d01aa 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/rfile/RFile.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/rfile/RFile.java
@@ -26,6 +26,7 @@
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.io.PrintStream;
+import java.io.UncheckedIOException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -36,6 +37,7 @@
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Objects;
 import java.util.Set;
 import java.util.TreeMap;
 import java.util.concurrent.atomic.AtomicBoolean;
@@ -55,6 +57,7 @@
 import org.apache.accumulo.core.file.FileSKVWriter;
 import org.apache.accumulo.core.file.NoSuchMetaStoreException;
 import org.apache.accumulo.core.file.blockfile.impl.CachableBlockFile;
+import org.apache.accumulo.core.file.blockfile.impl.CachableBlockFile.CachableBuilder;
 import org.apache.accumulo.core.file.blockfile.impl.CacheProvider;
 import org.apache.accumulo.core.file.rfile.BlockIndex.BlockIndexEntry;
 import org.apache.accumulo.core.file.rfile.MultiLevelIndex.IndexEntry;
@@ -72,16 +75,20 @@
 import org.apache.accumulo.core.iteratorsImpl.system.LocalityGroupIterator.LocalityGroup;
 import org.apache.accumulo.core.iteratorsImpl.system.LocalityGroupIterator.LocalityGroupContext;
 import org.apache.accumulo.core.iteratorsImpl.system.LocalityGroupIterator.LocalityGroupSeekCache;
+import org.apache.accumulo.core.metadata.TabletFile;
 import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
 import org.apache.accumulo.core.util.LocalityGroupUtil;
 import org.apache.accumulo.core.util.MutableByteSequence;
 import org.apache.commons.lang3.mutable.MutableLong;
+import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
+import com.google.common.base.Supplier;
+import com.google.common.base.Suppliers;
 
 public class RFile {
 
@@ -1034,8 +1041,8 @@
 
           if (iiter.hasPrevious()) {
             prevKey = new Key(iiter.peekPrevious().getKey()); // initially prevKey is the last key
+                                                              // of the prev block
           } else {
-            // of the prev block
             prevKey = new Key(); // first block in the file, so set prev key to minimal key
           }
 
@@ -1101,16 +1108,16 @@
     }
 
     @Override
-    public Key getFirstKey() {
-      return firstKey;
+    public Text getFirstRow() {
+      return firstKey != null ? firstKey.getRow() : null;
     }
 
     @Override
-    public Key getLastKey() {
+    public Text getLastRow() {
       if (index.size() == 0) {
         return null;
       }
-      return index.getLastKey();
+      return index.getLastKey().getRow();
     }
 
     @Override
@@ -1166,7 +1173,7 @@
     }
   }
 
-  public static class Reader extends HeapIterator implements FileSKVIterator {
+  public static class Reader extends HeapIterator implements RFileSKVIterator {
 
     private final CachableBlockFile.Reader reader;
 
@@ -1301,7 +1308,7 @@
     @Override
     public void closeDeepCopies() {
       if (deepCopy) {
-        throw new RuntimeException("Calling closeDeepCopies on a deep copy is not supported");
+        throw new IllegalStateException("Calling closeDeepCopies on a deep copy is not supported");
       }
 
       for (Reader deepCopy : deepCopies) {
@@ -1314,7 +1321,7 @@
     @Override
     public void close() throws IOException {
       if (deepCopy) {
-        throw new RuntimeException("Calling close on a deep copy is not supported");
+        throw new IllegalStateException("Calling close on a deep copy is not supported");
       }
 
       closeDeepCopies();
@@ -1340,47 +1347,47 @@
     }
 
     @Override
-    public Key getFirstKey() throws IOException {
+    public Text getFirstRow() throws IOException {
       if (currentReaders.length == 0) {
         return null;
       }
 
-      Key minKey = null;
+      Text minRow = null;
 
       for (LocalityGroupReader currentReader : currentReaders) {
-        if (minKey == null) {
-          minKey = currentReader.getFirstKey();
+        if (minRow == null) {
+          minRow = currentReader.getFirstRow();
         } else {
-          Key firstKey = currentReader.getFirstKey();
-          if (firstKey != null && firstKey.compareTo(minKey) < 0) {
-            minKey = firstKey;
+          Text firstRow = currentReader.getFirstRow();
+          if (firstRow != null && firstRow.compareTo(minRow) < 0) {
+            minRow = firstRow;
           }
         }
       }
 
-      return minKey;
+      return minRow;
     }
 
     @Override
-    public Key getLastKey() throws IOException {
+    public Text getLastRow() throws IOException {
       if (currentReaders.length == 0) {
         return null;
       }
 
-      Key maxKey = null;
+      Text maxRow = null;
 
       for (LocalityGroupReader currentReader : currentReaders) {
-        if (maxKey == null) {
-          maxKey = currentReader.getLastKey();
+        if (maxRow == null) {
+          maxRow = currentReader.getLastRow();
         } else {
-          Key lastKey = currentReader.getLastKey();
-          if (lastKey != null && lastKey.compareTo(maxKey) > 0) {
-            maxKey = lastKey;
+          Text lastRow = currentReader.getLastRow();
+          if (lastRow != null && lastRow.compareTo(maxRow) > 0) {
+            maxRow = lastRow;
           }
         }
       }
 
-      return maxKey;
+      return maxRow;
     }
 
     @Override
@@ -1393,7 +1400,7 @@
     }
 
     @Override
-    public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
+    public Reader deepCopy(IteratorEnvironment env) {
       if (env != null && env.isSamplingEnabled()) {
         SamplerConfiguration sc = env.getSamplerConfiguration();
         if (sc == null) {
@@ -1482,6 +1489,7 @@
       return (lgCache == null ? 0 : lgCache.getNumLGSeeked());
     }
 
+    @Override
     public FileSKVIterator getIndex() throws IOException {
 
       ArrayList<Iterator<IndexEntry>> indexes = new ArrayList<>();
@@ -1494,7 +1502,7 @@
     }
 
     @Override
-    public FileSKVIterator getSample(SamplerConfigurationImpl sampleConfig) {
+    public Reader getSample(SamplerConfigurationImpl sampleConfig) {
       requireNonNull(sampleConfig);
 
       if (this.samplerConfig != null && this.samplerConfig.equals(sampleConfig)) {
@@ -1540,11 +1548,12 @@
     @Override
     public void setInterruptFlag(AtomicBoolean flag) {
       if (deepCopy) {
-        throw new RuntimeException("Calling setInterruptFlag on a deep copy is not supported");
+        throw new IllegalStateException("Calling setInterruptFlag on a deep copy is not supported");
       }
 
       if (!deepCopies.isEmpty()) {
-        throw new RuntimeException("Setting interrupt flag after calling deep copy not supported");
+        throw new IllegalStateException(
+            "Setting interrupt flag after calling deep copy not supported");
       }
 
       setInterruptFlagInternal(flag);
@@ -1591,5 +1600,244 @@
 
       return totalEntries;
     }
+
+    @Override
+    public void reset() {
+      clear();
+    }
+  }
+
+  public interface RFileSKVIterator extends FileSKVIterator {
+    FileSKVIterator getIndex() throws IOException;
+
+    void reset();
+  }
+
+  static abstract class FencedFileSKVIterator implements FileSKVIterator {
+
+    private final FileSKVIterator reader;
+    protected final Range fence;
+    private final Key fencedStartKey;
+    private final Supplier<Key> fencedEndKey;
+
+    public FencedFileSKVIterator(FileSKVIterator reader, Range fence) {
+      this.reader = Objects.requireNonNull(reader);
+      this.fence = Objects.requireNonNull(fence);
+      this.fencedStartKey = fence.getStartKey();
+      this.fencedEndKey = Suppliers.memoize(() -> getEndKey(fence.getEndKey()));
+    }
+
+    @Override
+    public void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> options,
+        IteratorEnvironment env) throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public boolean hasTop() {
+      return reader.hasTop();
+    }
+
+    @Override
+    public void next() throws IOException {
+      reader.next();
+    }
+
+    @Override
+    public Key getTopKey() {
+      return reader.getTopKey();
+    }
+
+    @Override
+    public Value getTopValue() {
+      return reader.getTopValue();
+    }
+
+    @Override
+    public Text getFirstRow() throws IOException {
+      var row = reader.getFirstRow();
+      if (row != null && fence.beforeStartKey(new Key(row))) {
+        return fencedStartKey.getRow();
+      } else {
+        return row;
+      }
+    }
+
+    @Override
+    public Text getLastRow() throws IOException {
+      var row = reader.getLastRow();
+      if (row != null && fence.afterEndKey(new Key(row))) {
+        return fencedEndKey.get().getRow();
+      } else {
+        return row;
+      }
+    }
+
+    @Override
+    public boolean isRunningLowOnMemory() {
+      return reader.isRunningLowOnMemory();
+    }
+
+    @Override
+    public void setInterruptFlag(AtomicBoolean flag) {
+      reader.setInterruptFlag(flag);
+    }
+
+    @Override
+    public DataInputStream getMetaStore(String name) throws IOException {
+      return reader.getMetaStore(name);
+    }
+
+    @Override
+    public void closeDeepCopies() throws IOException {
+      reader.closeDeepCopies();
+    }
+
+    @Override
+    public void setCacheProvider(CacheProvider cacheProvider) {
+      reader.setCacheProvider(cacheProvider);
+    }
+
+    @Override
+    public void close() throws IOException {
+      reader.close();
+    }
+
+    private Key getEndKey(Key key) {
+      // If they key is infinite it will be null or if inclusive we can just use it as is
+      // as it would be the correct value for getLastKey()
+      if (fence.isInfiniteStopKey() || fence.isEndKeyInclusive()) {
+        return key;
+      }
+
+      // If exclusive we need to strip the last byte to get the last key that is part of the
+      // actual range to return
+      final byte[] ba = key.getRow().getBytes();
+      Preconditions.checkArgument(ba.length > 0 && ba[ba.length - 1] == (byte) 0x00);
+      byte[] fba = new byte[ba.length - 1];
+      System.arraycopy(ba, 0, fba, 0, ba.length - 1);
+
+      return new Key(fba);
+    }
+
+  }
+
+  static class FencedIndex extends FencedFileSKVIterator {
+    private final FileSKVIterator source;
+
+    public FencedIndex(FileSKVIterator source, Range seekFence) {
+      super(source, seekFence);
+      this.source = source;
+    }
+
+    @Override
+    public boolean hasTop() {
+      // this code filters out data because the rfile index iterators do not support seek
+
+      // If startKey is set then discard everything until we reach the start
+      // of the range
+      if (fence.getStartKey() != null) {
+
+        while (source.hasTop() && fence.beforeStartKey(source.getTopKey())) {
+          try {
+            source.next();
+          } catch (IOException e) {
+            throw new UncheckedIOException(e);
+          }
+        }
+      }
+
+      // If endKey is set then ensure that the current key is not passed the end of the range
+      return source.hasTop() && !fence.afterEndKey(source.getTopKey());
+    }
+
+    @Override
+    public long estimateOverlappingEntries(KeyExtent extent) throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public FileSKVIterator getSample(SamplerConfigurationImpl sampleConfig) {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void seek(Range range, Collection<ByteSequence> columnFamilies, boolean inclusive)
+        throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
+      throw new UnsupportedOperationException();
+    }
+  }
+
+  static class FencedReader extends FencedFileSKVIterator implements RFileSKVIterator {
+
+    private final Reader reader;
+
+    public FencedReader(Reader reader, Range seekFence) {
+      super(reader, seekFence);
+      this.reader = reader;
+    }
+
+    @Override
+    public void seek(Range range, Collection<ByteSequence> columnFamilies, boolean inclusive)
+        throws IOException {
+      reader.reset();
+
+      if (fence != null) {
+        range = fence.clip(range, true);
+        if (range == null) {
+          return;
+        }
+      }
+
+      reader.seek(range, columnFamilies, inclusive);
+    }
+
+    @Override
+    public FencedReader deepCopy(IteratorEnvironment env) {
+      return new FencedReader(reader.deepCopy(env), fence);
+    }
+
+    @Override
+    public FileSKVIterator getIndex() throws IOException {
+      return new FencedIndex(reader.getIndex(), fence);
+    }
+
+    @Override
+    public long estimateOverlappingEntries(KeyExtent c) throws IOException {
+      KeyExtent overlapping = c.clip(fence, true);
+      if (overlapping == null) {
+        return 0;
+      }
+      return reader.estimateOverlappingEntries(overlapping);
+    }
+
+    @Override
+    public FileSKVIterator getSample(SamplerConfigurationImpl sampleConfig) {
+      final Reader sample = reader.getSample(sampleConfig);
+      return sample != null ? new FencedReader(sample, fence) : null;
+    }
+
+    @Override
+    public void reset() {
+      reader.reset();
+    }
+  }
+
+  public static RFileSKVIterator getReader(final CachableBuilder cb, final TabletFile dataFile)
+      throws IOException {
+    final RFile.Reader reader = new RFile.Reader(Objects.requireNonNull(cb));
+    return dataFile.hasRange() ? new FencedReader(reader, dataFile.getRange()) : reader;
+  }
+
+  public static RFileSKVIterator getReader(final CachableBuilder cb, Range range)
+      throws IOException {
+    final RFile.Reader reader = new RFile.Reader(Objects.requireNonNull(cb));
+    return !range.isInfiniteStartKey() || !range.isInfiniteStopKey()
+        ? new FencedReader(reader, range) : reader;
   }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/file/rfile/RFileOperations.java b/core/src/main/java/org/apache/accumulo/core/file/rfile/RFileOperations.java
index 609c195..cf3e9f6 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/rfile/RFileOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/rfile/RFileOperations.java
@@ -33,14 +33,15 @@
 import org.apache.accumulo.core.file.FileSKVIterator;
 import org.apache.accumulo.core.file.FileSKVWriter;
 import org.apache.accumulo.core.file.blockfile.impl.CachableBlockFile.CachableBuilder;
+import org.apache.accumulo.core.file.rfile.RFile.RFileSKVIterator;
 import org.apache.accumulo.core.file.rfile.bcfile.BCFile;
+import org.apache.accumulo.core.metadata.TabletFile;
 import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
 import org.apache.accumulo.core.sample.impl.SamplerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -53,18 +54,18 @@
 
   private static final Collection<ByteSequence> EMPTY_CF_SET = Collections.emptySet();
 
-  private static RFile.Reader getReader(FileOptions options) throws IOException {
+  private static RFileSKVIterator getReader(FileOptions options) throws IOException {
     CachableBuilder cb = new CachableBuilder()
-        .fsPath(options.getFileSystem(), new Path(options.getFilename()), options.dropCacheBehind)
+        .fsPath(options.getFileSystem(), options.getFile().getPath(), options.dropCacheBehind)
         .conf(options.getConfiguration()).fileLen(options.getFileLenCache())
         .cacheProvider(options.cacheProvider).readLimiter(options.getRateLimiter())
         .cryptoService(options.getCryptoService());
-    return new RFile.Reader(cb);
+    return RFile.getReader(cb, options.getFile());
   }
 
   @Override
   protected long getFileSize(FileOptions options) throws IOException {
-    return options.getFileSystem().getFileStatus(new Path(options.getFilename())).getLen();
+    return options.getFileSystem().getFileStatus(options.getFile().getPath()).getLen();
   }
 
   @Override
@@ -74,7 +75,7 @@
 
   @Override
   protected FileSKVIterator openReader(FileOptions options) throws IOException {
-    RFile.Reader reader = getReader(options);
+    FileSKVIterator reader = getReader(options);
 
     if (options.isSeekToBeginning()) {
       reader.seek(new Range((Key) null, null), EMPTY_CF_SET, false);
@@ -85,7 +86,7 @@
 
   @Override
   protected FileSKVIterator openScanReader(FileOptions options) throws IOException {
-    RFile.Reader reader = getReader(options);
+    FileSKVIterator reader = getReader(options);
     reader.seek(options.getRange(), options.getColumnFamilies(), options.isRangeInclusive());
     return reader;
   }
@@ -133,25 +134,25 @@
       }
       int bufferSize = conf.getInt("io.file.buffer.size", 4096);
 
-      String file = options.getFilename();
+      TabletFile file = options.getFile();
       FileSystem fs = options.getFileSystem();
 
       if (options.dropCacheBehind) {
         EnumSet<CreateFlag> set = EnumSet.of(CreateFlag.SYNC_BLOCK, CreateFlag.CREATE);
-        outputStream = fs.create(new Path(file), FsPermission.getDefault(), set, bufferSize,
+        outputStream = fs.create(file.getPath(), FsPermission.getDefault(), set, bufferSize,
             (short) rep, block, null);
         try {
           // Tell the DataNode that the file does not need to be cached in the OS page cache
           outputStream.setDropBehind(Boolean.TRUE);
-          LOG.trace("Called setDropBehind(TRUE) for stream writing file {}", options.filename);
+          LOG.trace("Called setDropBehind(TRUE) for stream writing file {}", options.file);
         } catch (UnsupportedOperationException e) {
-          LOG.debug("setDropBehind not enabled for file: {}", options.filename);
+          LOG.debug("setDropBehind not enabled for file: {}", options.file);
         } catch (IOException e) {
-          LOG.debug("IOException setting drop behind for file: {}, msg: {}", options.filename,
+          LOG.debug("IOException setting drop behind for file: {}, msg: {}", options.file,
               e.getMessage());
         }
       } else {
-        outputStream = fs.create(new Path(file), false, bufferSize, (short) rep, block);
+        outputStream = fs.create(file.getPath(), false, bufferSize, (short) rep, block);
       }
     }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/CompressionAlgorithm.java b/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/CompressionAlgorithm.java
index a82dd8a..f76983c 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/CompressionAlgorithm.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/CompressionAlgorithm.java
@@ -25,7 +25,6 @@
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.util.Map.Entry;
-import java.util.concurrent.ExecutionException;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.accumulo.core.spi.file.rfile.compression.CompressionAlgorithmConfiguration;
@@ -41,9 +40,8 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.cache.CacheBuilder;
-import com.google.common.cache.CacheLoader;
-import com.google.common.cache.LoadingCache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.LoadingCache;
 import com.google.common.collect.Maps;
 
 /**
@@ -113,12 +111,8 @@
    * Guava cache to have a limited factory pattern defined in the Algorithm enum.
    */
   private static LoadingCache<Entry<CompressionAlgorithm,Integer>,CompressionCodec> codecCache =
-      CacheBuilder.newBuilder().maximumSize(25).build(new CacheLoader<>() {
-        @Override
-        public CompressionCodec load(Entry<CompressionAlgorithm,Integer> key) {
-          return key.getKey().createNewCodec(key.getValue());
-        }
-      });
+      Caffeine.newBuilder().maximumSize(25)
+          .build(key -> key.getKey().createNewCodec(key.getValue()));
 
   // Data input buffer size to absorb small reads from application.
   protected static final int DATA_IBUF_SIZE = 1024;
@@ -170,11 +164,7 @@
     // If the default buffer size is not being used, pull from the loading cache.
     if (bufferSize != defaultBufferSize) {
       Entry<CompressionAlgorithm,Integer> sizeOpt = Maps.immutableEntry(algorithm, bufferSize);
-      try {
-        codec = codecCache.get(sizeOpt);
-      } catch (ExecutionException e) {
-        throw new IOException(e);
-      }
+      codec = codecCache.get(sizeOpt);
     }
     CompressionInputStream cis = codec.createInputStream(stream, decompressor);
     return new BufferedInputStream(cis, DATA_IBUF_SIZE);
diff --git a/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/PrintBCInfo.java b/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/PrintBCInfo.java
index fd8356a..55d4338 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/PrintBCInfo.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/PrintBCInfo.java
@@ -47,8 +47,7 @@
   CryptoService cryptoService = NoCryptoServiceFactory.NONE;
 
   public void printMetaBlockInfo() throws IOException {
-    FSDataInputStream fsin = fs.open(path);
-    try (BCFile.Reader bcfr =
+    try (FSDataInputStream fsin = fs.open(path); BCFile.Reader bcfr =
         new BCFile.Reader(fsin, fs.getFileStatus(path).getLen(), conf, cryptoService)) {
 
       Set<Entry<String,MetaIndexEntry>> es = bcfr.metaIndex.index.entrySet();
@@ -67,6 +66,17 @@
     }
   }
 
+  public String getCompressionType() throws IOException {
+    try (FSDataInputStream fsin = fs.open(path); BCFile.Reader bcfr =
+        new BCFile.Reader(fsin, fs.getFileStatus(path).getLen(), conf, cryptoService)) {
+
+      Set<Entry<String,MetaIndexEntry>> es = bcfr.metaIndex.index.entrySet();
+
+      return es.stream().filter(entry -> entry.getKey().equals("RFile.index")).findFirst()
+          .map(entry -> entry.getValue().getCompressionAlgorithm().getName()).orElse(null);
+    }
+  }
+
   static class Opts extends ConfigOpts {
     @Parameter(description = " <file>")
     String file;
diff --git a/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/Utils.java b/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/Utils.java
index d3fbfac..3f96630 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/Utils.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/Utils.java
@@ -142,7 +142,7 @@
         out.writeLong(n);
         return;
       default:
-        throw new RuntimeException("Internal error");
+        throw new IllegalStateException("Internal error");
     }
   }
 
@@ -157,7 +157,7 @@
   public static int readVInt(DataInput in) throws IOException {
     long ret = readVLong(in);
     if ((ret > Integer.MAX_VALUE) || (ret < Integer.MIN_VALUE)) {
-      throw new RuntimeException("Number too large to be represented as Integer");
+      throw new IllegalStateException("Number too large to be represented as Integer");
     }
     return (int) ret;
   }
@@ -218,7 +218,7 @@
             throw new IOException("Corrupted VLong encoding");
         }
       default:
-        throw new RuntimeException("Internal error");
+        throw new IllegalStateException("Internal error");
     }
   }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/gc/Reference.java b/core/src/main/java/org/apache/accumulo/core/gc/Reference.java
index 4c67bfd..a6ee689 100644
--- a/core/src/main/java/org/apache/accumulo/core/gc/Reference.java
+++ b/core/src/main/java/org/apache/accumulo/core/gc/Reference.java
@@ -42,13 +42,14 @@
   TableId getTableId();
 
   /**
-   * Get the exact string stored in the metadata table for this file or directory. A file will be
-   * read from the Tablet "file" column family:
+   * Get the path stored in the metadata table for this file or directory. The path will be read
+   * from the Tablet "file" column family:
    * {@link org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily}
    * A directory will be read from the "srv:dir" column family:
    * {@link org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily}
    * A scan will be read from the Tablet "scan" column family:
    * {@link org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ScanFileColumnFamily}
    */
-  String getMetadataEntry();
+  String getMetadataPath();
+
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/gc/ReferenceDirectory.java b/core/src/main/java/org/apache/accumulo/core/gc/ReferenceDirectory.java
index 5491020..a3bff2b 100644
--- a/core/src/main/java/org/apache/accumulo/core/gc/ReferenceDirectory.java
+++ b/core/src/main/java/org/apache/accumulo/core/gc/ReferenceDirectory.java
@@ -46,11 +46,11 @@
    * A Tablet directory should have a metadata entry equal to the dirName.
    */
   @Override
-  public String getMetadataEntry() {
-    if (!tabletDir.equals(metadataEntry)) {
+  public String getMetadataPath() {
+    if (!tabletDir.equals(metadataPath)) {
       throw new IllegalStateException(
-          "Tablet dir " + tabletDir + " is not equal to metadataEntry: " + metadataEntry);
+          "Tablet dir " + tabletDir + " is not equal to metadataPath: " + metadataPath);
     }
-    return metadataEntry;
+    return metadataPath;
   }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/gc/ReferenceFile.java b/core/src/main/java/org/apache/accumulo/core/gc/ReferenceFile.java
index b9eece9..3044ec2 100644
--- a/core/src/main/java/org/apache/accumulo/core/gc/ReferenceFile.java
+++ b/core/src/main/java/org/apache/accumulo/core/gc/ReferenceFile.java
@@ -21,6 +21,9 @@
 import java.util.Objects;
 
 import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.metadata.ScanServerRefTabletFile;
+import org.apache.accumulo.core.metadata.StoredTabletFile;
+import org.apache.hadoop.fs.Path;
 
 /**
  * A GC reference used for streaming and delete markers. This type is a file. Subclass is a
@@ -31,21 +34,33 @@
   public final TableId tableId; // 2a
   public final boolean isScan;
 
-  // the exact string that is stored in the metadata
-  protected final String metadataEntry;
+  // the exact path from the file reference string that is stored in the metadata
+  protected final String metadataPath;
 
-  protected ReferenceFile(TableId tableId, String metadataEntry, boolean isScan) {
+  protected ReferenceFile(TableId tableId, String metadataPath, boolean isScan) {
     this.tableId = Objects.requireNonNull(tableId);
-    this.metadataEntry = Objects.requireNonNull(metadataEntry);
+    this.metadataPath = Objects.requireNonNull(metadataPath);
     this.isScan = isScan;
   }
 
-  public static ReferenceFile forFile(TableId tableId, String metadataEntry) {
-    return new ReferenceFile(tableId, metadataEntry, false);
+  public static ReferenceFile forFile(TableId tableId, StoredTabletFile tabletFile) {
+    return new ReferenceFile(tableId, tabletFile.getMetadataPath(), false);
   }
 
-  public static ReferenceFile forScan(TableId tableId, String metadataEntry) {
-    return new ReferenceFile(tableId, metadataEntry, true);
+  public static ReferenceFile forFile(TableId tableId, Path metadataPathPath) {
+    return new ReferenceFile(tableId, metadataPathPath.toString(), false);
+  }
+
+  public static ReferenceFile forScan(TableId tableId, ScanServerRefTabletFile tabletFile) {
+    return new ReferenceFile(tableId, tabletFile.getNormalizedPathStr(), true);
+  }
+
+  public static ReferenceFile forScan(TableId tableId, StoredTabletFile tabletFile) {
+    return new ReferenceFile(tableId, tabletFile.getMetadataPath(), true);
+  }
+
+  public static ReferenceFile forScan(TableId tableId, Path metadataPathPath) {
+    return new ReferenceFile(tableId, metadataPathPath.toString(), true);
   }
 
   @Override
@@ -64,8 +79,8 @@
   }
 
   @Override
-  public String getMetadataEntry() {
-    return metadataEntry;
+  public String getMetadataPath() {
+    return metadataPath;
   }
 
   @Override
@@ -73,7 +88,7 @@
     if (equals(that)) {
       return 0;
     } else {
-      return this.metadataEntry.compareTo(that.metadataEntry);
+      return this.metadataPath.compareTo(that.metadataPath);
     }
   }
 
@@ -89,17 +104,17 @@
       return false;
     }
     ReferenceFile other = (ReferenceFile) obj;
-    return metadataEntry.equals(other.metadataEntry);
+    return metadataPath.equals(other.metadataPath);
   }
 
   @Override
   public int hashCode() {
-    return this.metadataEntry.hashCode();
+    return this.metadataPath.hashCode();
   }
 
   @Override
   public String toString() {
-    return "Reference [id=" + tableId + ", ref=" + metadataEntry + "]";
+    return "Reference [id=" + tableId + ", ref=" + metadataPath + "]";
   }
 
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/Combiner.java b/core/src/main/java/org/apache/accumulo/core/iterators/Combiner.java
index 40922bd..b87f309 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/Combiner.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/Combiner.java
@@ -27,7 +27,6 @@
 import java.util.List;
 import java.util.Map;
 import java.util.NoSuchElementException;
-import java.util.concurrent.ExecutionException;
 
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.IteratorSetting.Column;
@@ -43,10 +42,10 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Splitter;
-import com.google.common.cache.Cache;
-import com.google.common.cache.CacheBuilder;
 import com.google.common.collect.Lists;
 
 /**
@@ -187,12 +186,11 @@
 
   @VisibleForTesting
   static final Cache<String,Boolean> loggedMsgCache =
-      CacheBuilder.newBuilder().expireAfterWrite(1, HOURS).maximumSize(10000).build();
+      Caffeine.newBuilder().expireAfterWrite(1, HOURS).maximumSize(10000).build();
 
   private void sawDelete() {
-    if (isMajorCompaction && !reduceOnFullCompactionOnly) {
-      try {
-        loggedMsgCache.get(this.getClass().getName(), () -> {
+    if (isMajorCompaction && !reduceOnFullCompactionOnly
+        && loggedMsgCache.get(this.getClass().getName(), k -> {
           sawDeleteLog.error(
               "Combiner of type {} saw a delete during a"
                   + " partial compaction. This could cause undesired results. See"
@@ -200,10 +198,11 @@
               Combiner.this.getClass().getSimpleName());
           // the value is not used and does not matter
           return Boolean.TRUE;
-        });
-      } catch (ExecutionException e) {
-        throw new RuntimeException(e);
-      }
+        })) {
+      // do nothing;
+      // this is a workaround to ignore the return value of the cache, since we're relying only on
+      // the side-effect of logging when the cache entry expires;
+      // if the cached value is present, it's value is always true
     }
   }
 
@@ -316,8 +315,8 @@
     Combiner newInstance;
     try {
       newInstance = this.getClass().getDeclaredConstructor().newInstance();
-    } catch (Exception e) {
-      throw new RuntimeException(e);
+    } catch (ReflectiveOperationException e) {
+      throw new IllegalStateException(e);
     }
     newInstance.setSource(getSource().deepCopy(env));
     newInstance.combiners = combiners;
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/Filter.java b/core/src/main/java/org/apache/accumulo/core/iterators/Filter.java
index 9c12dcc..8492fb8 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/Filter.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/Filter.java
@@ -19,6 +19,7 @@
 package org.apache.accumulo.core.iterators;
 
 import java.io.IOException;
+import java.io.UncheckedIOException;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Map;
@@ -45,8 +46,8 @@
     Filter newInstance;
     try {
       newInstance = this.getClass().getDeclaredConstructor().newInstance();
-    } catch (Exception e) {
-      throw new RuntimeException(e);
+    } catch (ReflectiveOperationException e) {
+      throw new IllegalStateException(e);
     }
     newInstance.setSource(getSource().deepCopy(env));
     newInstance.negate = negate;
@@ -79,7 +80,7 @@
       try {
         source.next();
       } catch (IOException e) {
-        throw new RuntimeException(e);
+        throw new UncheckedIOException(e);
       }
     }
   }
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/IteratorEnvironment.java b/core/src/main/java/org/apache/accumulo/core/iterators/IteratorEnvironment.java
index 372a0e4..0e42acb 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/IteratorEnvironment.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/IteratorEnvironment.java
@@ -18,40 +18,16 @@
  */
 package org.apache.accumulo.core.iterators;
 
-import java.io.IOException;
-
 import org.apache.accumulo.core.client.PluginEnvironment;
 import org.apache.accumulo.core.client.SampleNotPresentException;
 import org.apache.accumulo.core.client.sample.SamplerConfiguration;
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.spi.common.ServiceEnvironment;
 
 public interface IteratorEnvironment {
 
   /**
-   * @deprecated since 2.0.0. This is a legacy method used for internal backwards compatibility.
-   */
-  @Deprecated(since = "2.0.0")
-  default SortedKeyValueIterator<Key,Value> reserveMapFileReader(String mapFileName)
-      throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  /**
-   * @deprecated since 2.0.0. This method was using an unstable non public type. Use
-   *             {@link #getPluginEnv()}
-   */
-  @Deprecated(since = "2.0.0")
-  default AccumuloConfiguration getConfig() {
-    throw new UnsupportedOperationException();
-  }
-
-  /**
    * Return the executed scope of the Iterator. Value will be one of the following:
    * {@link IteratorScope#scan}, {@link IteratorScope#minc}, {@link IteratorScope#majc}
    */
@@ -68,14 +44,6 @@
   }
 
   /**
-   * @deprecated since 2.0.0. This was an experimental feature and was never tested or documented.
-   */
-  @Deprecated(since = "2.0.0")
-  default void registerSideChannel(SortedKeyValueIterator<Key,Value> iter) {
-    throw new UnsupportedOperationException();
-  }
-
-  /**
    * Return the Scan Authorizations used in this Iterator. Will throw UnsupportedOperationException
    * if {@link #getIteratorScope()} != {@link IteratorScope#scan}.
    */
@@ -153,30 +121,13 @@
    * obtain a table configuration, use the following methods:
    *
    * <pre>
-   * iterEnv.getServiceEnv().getConfiguration(env.getTableId())
-   * </pre>
-   *
-   * @since 2.0.0
-   * @deprecated since 2.1.0. This method was using a non public API type. Use
-   *             {@link #getPluginEnv()} instead because it has better stability guarantees.
-   */
-  @Deprecated(since = "2.1.0")
-  default ServiceEnvironment getServiceEnv() {
-    throw new UnsupportedOperationException();
-  }
-
-  /**
-   * Returns an object containing information about the server where this iterator was run. To
-   * obtain a table configuration, use the following methods:
-   *
-   * <pre>
    * iterEnv.getPluginEnv().getConfiguration(env.getTableId())
    * </pre>
    *
    * @since 2.1.0
    */
   default PluginEnvironment getPluginEnv() {
-    return getServiceEnv();
+    throw new UnsupportedOperationException();
   }
 
   /**
@@ -187,4 +138,14 @@
   default TableId getTableId() {
     throw new UnsupportedOperationException();
   }
+
+  /**
+   * Return whether or not the server is running low on memory
+   *
+   * @return true if server is running low on memory
+   * @since 3.0.0
+   */
+  default boolean isRunningLowOnMemory() {
+    throw new UnsupportedOperationException();
+  }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/SortedKeyValueIterator.java b/core/src/main/java/org/apache/accumulo/core/iterators/SortedKeyValueIterator.java
index be9bfe0..30efd5f 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/SortedKeyValueIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/SortedKeyValueIterator.java
@@ -149,4 +149,17 @@
    * @exception UnsupportedOperationException if not supported.
    */
   SortedKeyValueIterator<K,V> deepCopy(IteratorEnvironment env);
+
+  /**
+   * Returns true when running in a server process and the GarbageCollectionLogger determines that
+   * the server is running low on memory. This is useful for iterators that aggregate KV pairs or
+   * perform long running operations that create a lot of garbage. Server side iterators can
+   * override this method and return the value of IteratorEnvironment.isRunningLowOnMemory.
+   *
+   * @return true if running in server process and server is running low on memory
+   * @since 3.0.0
+   */
+  default boolean isRunningLowOnMemory() {
+    return false;
+  }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/WrappingIterator.java b/core/src/main/java/org/apache/accumulo/core/iterators/WrappingIterator.java
index eb84bdc..f7edbb4 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/WrappingIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/WrappingIterator.java
@@ -43,6 +43,7 @@
 public abstract class WrappingIterator implements SortedKeyValueIterator<Key,Value> {
 
   private SortedKeyValueIterator<Key,Value> source = null;
+  private IteratorEnvironment env = null;
   boolean seenSeek = false;
 
   protected void setSource(SortedKeyValueIterator<Key,Value> source) {
@@ -89,7 +90,7 @@
   public void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> options,
       IteratorEnvironment env) throws IOException {
     this.setSource(source);
-
+    this.env = env;
   }
 
   @Override
@@ -107,4 +108,12 @@
     seenSeek = true;
   }
 
+  @Override
+  public boolean isRunningLowOnMemory() {
+    if (env == null) {
+      return SortedKeyValueIterator.super.isRunningLowOnMemory();
+    }
+    return env.isRunningLowOnMemory();
+  }
+
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/user/RowEncodingIterator.java b/core/src/main/java/org/apache/accumulo/core/iterators/user/RowEncodingIterator.java
index 27c269a..79f81d8 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/user/RowEncodingIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/user/RowEncodingIterator.java
@@ -90,8 +90,8 @@
     RowEncodingIterator newInstance;
     try {
       newInstance = this.getClass().getDeclaredConstructor().newInstance();
-    } catch (Exception e) {
-      throw new RuntimeException(e);
+    } catch (ReflectiveOperationException e) {
+      throw new IllegalStateException(e);
     }
     newInstance.sourceIter = sourceIter.deepCopy(env);
     newInstance.maxBufferSize = maxBufferSize;
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/user/RowFilter.java b/core/src/main/java/org/apache/accumulo/core/iterators/user/RowFilter.java
index 54d8b09..ac93ab1 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/user/RowFilter.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/user/RowFilter.java
@@ -19,6 +19,7 @@
 package org.apache.accumulo.core.iterators.user;
 
 import java.io.IOException;
+import java.io.UncheckedIOException;
 import java.util.Collection;
 import java.util.Map;
 
@@ -158,8 +159,10 @@
     try {
       newInstance = getClass().getDeclaredConstructor().newInstance();
       newInstance.init(getSource().deepCopy(env), options, env);
-    } catch (Exception e) {
-      throw new RuntimeException(e);
+    } catch (ReflectiveOperationException e) {
+      throw new IllegalStateException(e);
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
     }
     newInstance.decisionIterator = new RowIterator(getSource().deepCopy(env));
     return newInstance;
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/user/SeekingFilter.java b/core/src/main/java/org/apache/accumulo/core/iterators/user/SeekingFilter.java
index d6094a9..d3bcee8 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/user/SeekingFilter.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/user/SeekingFilter.java
@@ -152,8 +152,8 @@
     SeekingFilter newInstance;
     try {
       newInstance = this.getClass().getDeclaredConstructor().newInstance();
-    } catch (Exception e) {
-      throw new RuntimeException(e);
+    } catch (ReflectiveOperationException e) {
+      throw new IllegalStateException(e);
     }
     newInstance.setSource(getSource().deepCopy(env));
     newInstance.negate = negate;
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/user/TransformingIterator.java b/core/src/main/java/org/apache/accumulo/core/iterators/user/TransformingIterator.java
index 4329c6d..492ef6d 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/user/TransformingIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/user/TransformingIterator.java
@@ -30,6 +30,9 @@
 import java.util.Map.Entry;
 import java.util.NoSuchElementException;
 
+import org.apache.accumulo.access.AccessEvaluator;
+import org.apache.accumulo.access.AccessExpression;
+import org.apache.accumulo.access.InvalidAccessExpressionException;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.conf.ConfigurationTypeHelper;
 import org.apache.accumulo.core.data.ByteSequence;
@@ -43,10 +46,6 @@
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 import org.apache.accumulo.core.iterators.WrappingIterator;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.ColumnVisibility;
-import org.apache.accumulo.core.security.VisibilityEvaluator;
-import org.apache.accumulo.core.security.VisibilityParseException;
-import org.apache.accumulo.core.util.BadArgumentException;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.commons.collections4.map.LRUMap;
 import org.apache.hadoop.io.Text;
@@ -103,7 +102,7 @@
   protected Collection<ByteSequence> seekColumnFamilies;
   protected boolean seekColumnFamiliesInclusive;
 
-  private VisibilityEvaluator ve = null;
+  private AccessEvaluator ve = null;
   private LRUMap<ByteSequence,Boolean> visibleCache = null;
   private LRUMap<ByteSequence,Boolean> parsedVisibilitiesCache = null;
   private long maxBufferSize;
@@ -118,7 +117,7 @@
     if (scanning) {
       String auths = options.get(AUTH_OPT);
       if (auths != null && !auths.isEmpty()) {
-        ve = new VisibilityEvaluator(new Authorizations(auths.getBytes(UTF_8)));
+        ve = AccessEvaluator.of(new Authorizations(auths.getBytes(UTF_8)).toAccessAuthorizations());
         visibleCache = new LRUMap<>(100);
       }
     }
@@ -176,8 +175,8 @@
 
     try {
       copy = getClass().getDeclaredConstructor().newInstance();
-    } catch (Exception e) {
-      throw new RuntimeException(e);
+    } catch (ReflectiveOperationException e) {
+      throw new IllegalStateException(e);
     }
 
     copy.setSource(getSource().deepCopy(env));
@@ -409,13 +408,12 @@
     // Ensure that the visibility (which could have been transformed) parses. Must always do this
     // check, even if visibility is not evaluated.
     ByteSequence visibility = key.getColumnVisibilityData();
-    ColumnVisibility colVis = null;
     Boolean parsed = parsedVisibilitiesCache.get(visibility);
     if (parsed == null) {
       try {
-        colVis = new ColumnVisibility(visibility.toArray());
+        AccessExpression.validate(visibility.toArray());
         parsedVisibilitiesCache.put(visibility, Boolean.TRUE);
-      } catch (BadArgumentException e) {
+      } catch (InvalidAccessExpressionException e) {
         log.error("Parse error after transformation : {}", visibility);
         parsedVisibilitiesCache.put(visibility, Boolean.FALSE);
         if (scanning) {
@@ -441,12 +439,9 @@
     visible = visibleCache.get(visibility);
     if (visible == null) {
       try {
-        if (colVis == null) {
-          colVis = new ColumnVisibility(visibility.toArray());
-        }
-        visible = ve.evaluate(colVis);
+        visible = ve.canAccess(visibility.toArray());
         visibleCache.put(visibility, visible);
-      } catch (VisibilityParseException | BadArgumentException e) {
+      } catch (InvalidAccessExpressionException e) {
         log.error("Parse Error", e);
         visible = Boolean.FALSE;
       }
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/user/VisibilityFilter.java b/core/src/main/java/org/apache/accumulo/core/iterators/user/VisibilityFilter.java
index c91009b..3bbb77f 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/user/VisibilityFilter.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/user/VisibilityFilter.java
@@ -23,6 +23,9 @@
 import java.io.IOException;
 import java.util.Map;
 
+import org.apache.accumulo.access.AccessEvaluator;
+import org.apache.accumulo.access.AccessExpression;
+import org.apache.accumulo.access.InvalidAccessExpressionException;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.data.Key;
@@ -32,10 +35,6 @@
 import org.apache.accumulo.core.iterators.OptionDescriber;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.ColumnVisibility;
-import org.apache.accumulo.core.security.VisibilityEvaluator;
-import org.apache.accumulo.core.security.VisibilityParseException;
-import org.apache.accumulo.core.util.BadArgumentException;
 import org.apache.commons.collections4.map.LRUMap;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -45,7 +44,7 @@
  */
 public class VisibilityFilter extends Filter implements OptionDescriber {
 
-  protected VisibilityEvaluator ve;
+  private AccessEvaluator accessEvaluator;
   protected Map<ByteSequence,Boolean> cache;
 
   private static final Logger log = LoggerFactory.getLogger(VisibilityFilter.class);
@@ -66,7 +65,8 @@
       String auths = options.get(AUTHS);
       Authorizations authObj = auths == null || auths.isEmpty() ? new Authorizations()
           : new Authorizations(auths.getBytes(UTF_8));
-      this.ve = new VisibilityEvaluator(authObj);
+
+      this.accessEvaluator = AccessEvaluator.of(authObj.toAccessAuthorizations());
     }
     this.cache = new LRUMap<>(1000);
   }
@@ -80,10 +80,10 @@
         return b;
       }
       try {
-        new ColumnVisibility(testVis.toArray());
+        AccessExpression.validate(testVis.toArray());
         cache.put(testVis, true);
         return true;
-      } catch (BadArgumentException e) {
+      } catch (InvalidAccessExpressionException e) {
         cache.put(testVis, false);
         return false;
       }
@@ -98,10 +98,10 @@
       }
 
       try {
-        boolean bb = ve.evaluate(new ColumnVisibility(testVis.toArray()));
+        boolean bb = accessEvaluator.canAccess(testVis.toArray());
         cache.put(testVis, bb);
         return bb;
-      } catch (VisibilityParseException | BadArgumentException e) {
+      } catch (InvalidAccessExpressionException e) {
         log.error("Parse Error", e);
         return false;
       }
diff --git a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorConfigUtil.java b/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorConfigUtil.java
index 93c098d..7149995 100644
--- a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorConfigUtil.java
+++ b/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorConfigUtil.java
@@ -199,8 +199,9 @@
     final boolean useClassLoader = iteratorBuilder.useAccumuloClassLoader;
     Map<String,Class<SortedKeyValueIterator<Key,Value>>> classCache = new HashMap<>();
 
-    for (IterInfo iterInfo : iteratorBuilder.iters) {
-      try {
+    try {
+      for (IterInfo iterInfo : iteratorBuilder.iters) {
+
         Class<SortedKeyValueIterator<Key,Value>> clazz = null;
         log.trace("Attempting to load iterator class {}", iterInfo.className);
         if (iteratorBuilder.useClassCache) {
@@ -224,12 +225,10 @@
 
         skvi.init(prev, options, iteratorBuilder.iteratorEnvironment);
         prev = skvi;
-
-      } catch (ReflectiveOperationException e) {
-        log.error("Failed to load iterator {}, for table {}, from context {}", iterInfo.className,
-            iteratorBuilder.iteratorEnvironment.getTableId(), iteratorBuilder.context, e);
-        throw new RuntimeException(e);
       }
+    } catch (ReflectiveOperationException e) {
+      log.error(e.toString());
+      throw new IllegalStateException(e);
     }
     return prev;
   }
diff --git a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/system/MapFileIterator.java b/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/system/MapFileIterator.java
deleted file mode 100644
index 3f84a3a..0000000
--- a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/system/MapFileIterator.java
+++ /dev/null
@@ -1,128 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.iteratorsImpl.system;
-
-import java.io.DataInputStream;
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Map;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import org.apache.accumulo.core.data.ByteSequence;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.dataImpl.KeyExtent;
-import org.apache.accumulo.core.file.FileSKVIterator;
-import org.apache.accumulo.core.file.blockfile.impl.CacheProvider;
-import org.apache.accumulo.core.iterators.IteratorEnvironment;
-import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
-import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-
-public class MapFileIterator implements FileSKVIterator {
-
-  private static final String MSG = "Map files are not supported";
-
-  public MapFileIterator(FileSystem fs, String dir, Configuration conf) {
-    throw new UnsupportedOperationException(MSG);
-  }
-
-  @Override
-  public void setInterruptFlag(AtomicBoolean flag) {
-    throw new UnsupportedOperationException(MSG);
-  }
-
-  @Override
-  public void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> options,
-      IteratorEnvironment env) {
-    throw new UnsupportedOperationException(MSG);
-  }
-
-  @Override
-  public boolean hasTop() {
-    throw new UnsupportedOperationException(MSG);
-  }
-
-  @Override
-  public void next() {
-    throw new UnsupportedOperationException(MSG);
-  }
-
-  @Override
-  public void seek(Range range, Collection<ByteSequence> columnFamilies, boolean inclusive) {
-    throw new UnsupportedOperationException(MSG);
-  }
-
-  @Override
-  public Key getTopKey() {
-    throw new UnsupportedOperationException(MSG);
-  }
-
-  @Override
-  public Value getTopValue() {
-    throw new UnsupportedOperationException(MSG);
-  }
-
-  @Override
-  public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
-    throw new UnsupportedOperationException(MSG);
-  }
-
-  @Override
-  public Key getFirstKey() {
-    throw new UnsupportedOperationException(MSG);
-  }
-
-  @Override
-  public Key getLastKey() {
-    throw new UnsupportedOperationException(MSG);
-  }
-
-  @Override
-  public DataInputStream getMetaStore(String name) {
-    throw new UnsupportedOperationException(MSG);
-  }
-
-  @Override
-  public long estimateOverlappingEntries(KeyExtent extent) throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void closeDeepCopies() {
-    throw new UnsupportedOperationException(MSG);
-  }
-
-  @Override
-  public void close() {
-    throw new UnsupportedOperationException(MSG);
-  }
-
-  @Override
-  public FileSKVIterator getSample(SamplerConfigurationImpl sampleConfig) {
-    throw new UnsupportedOperationException(MSG);
-  }
-
-  @Override
-  public void setCacheProvider(CacheProvider cacheProvider) {
-    throw new UnsupportedOperationException(MSG);
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/system/SequenceFileIterator.java b/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/system/SequenceFileIterator.java
index 474255a..71cbb76 100644
--- a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/system/SequenceFileIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/system/SequenceFileIterator.java
@@ -36,6 +36,7 @@
 import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.SequenceFile.Reader;
+import org.apache.hadoop.io.Text;
 
 public class SequenceFileIterator implements FileSKVIterator {
 
@@ -117,12 +118,12 @@
   }
 
   @Override
-  public Key getFirstKey() throws IOException {
+  public Text getFirstRow() throws IOException {
     throw new UnsupportedOperationException("getFirstKey() not supported");
   }
 
   @Override
-  public Key getLastKey() throws IOException {
+  public Text getLastRow() throws IOException {
     throw new UnsupportedOperationException("getLastKey() not supported");
   }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/system/SystemIteratorUtil.java b/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/system/SystemIteratorUtil.java
index bac1d31..cfad4ee 100644
--- a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/system/SystemIteratorUtil.java
+++ b/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/system/SystemIteratorUtil.java
@@ -32,10 +32,6 @@
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.tabletserver.thrift.IteratorConfig;
 import org.apache.accumulo.core.tabletserver.thrift.TIteratorSetting;
-import org.apache.thrift.TDeserializer;
-import org.apache.thrift.TException;
-import org.apache.thrift.TSerializer;
-import org.apache.thrift.protocol.TBinaryProtocol;
 
 /**
  * System utility class. Not for client use.
@@ -62,39 +58,6 @@
     return new IteratorConfig(tisList);
   }
 
-  public static List<IteratorSetting> toIteratorSettings(IteratorConfig ic) {
-    List<IteratorSetting> ret = new ArrayList<>();
-    for (TIteratorSetting tIteratorSetting : ic.getIterators()) {
-      ret.add(toIteratorSetting(tIteratorSetting));
-    }
-
-    return ret;
-  }
-
-  public static byte[] encodeIteratorSettings(IteratorConfig iterators) {
-    try {
-      TSerializer tser = new TSerializer(new TBinaryProtocol.Factory());
-      return tser.serialize(iterators);
-    } catch (TException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  public static byte[] encodeIteratorSettings(List<IteratorSetting> iterators) {
-    return encodeIteratorSettings(toIteratorConfig(iterators));
-  }
-
-  public static List<IteratorSetting> decodeIteratorSettings(byte[] enc) {
-    IteratorConfig ic = new IteratorConfig();
-    try {
-      TDeserializer tdser = new TDeserializer(new TBinaryProtocol.Factory());
-      tdser.deserialize(ic, enc);
-    } catch (TException e) {
-      throw new RuntimeException(e);
-    }
-    return toIteratorSettings(ic);
-  }
-
   public static SortedKeyValueIterator<Key,Value> setupSystemScanIterators(
       SortedKeyValueIterator<Key,Value> source, Set<Column> cols, Authorizations auths,
       byte[] defaultVisibility, AccumuloConfiguration conf) throws IOException {
diff --git a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/system/VisibilityFilter.java b/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/system/VisibilityFilter.java
index 0a0fd5f..ddb60ca 100644
--- a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/system/VisibilityFilter.java
+++ b/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/system/VisibilityFilter.java
@@ -18,6 +18,8 @@
  */
 package org.apache.accumulo.core.iteratorsImpl.system;
 
+import org.apache.accumulo.access.AccessEvaluator;
+import org.apache.accumulo.access.InvalidAccessExpressionException;
 import org.apache.accumulo.core.data.ArrayByteSequence;
 import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.data.Key;
@@ -26,10 +28,6 @@
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 import org.apache.accumulo.core.iterators.SynchronizedServerFilter;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.ColumnVisibility;
-import org.apache.accumulo.core.security.VisibilityEvaluator;
-import org.apache.accumulo.core.security.VisibilityParseException;
-import org.apache.accumulo.core.util.BadArgumentException;
 import org.apache.commons.collections4.map.LRUMap;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -43,7 +41,7 @@
  * class.
  */
 public class VisibilityFilter extends SynchronizedServerFilter {
-  protected VisibilityEvaluator ve;
+  protected AccessEvaluator ve;
   protected ByteSequence defaultVisibility;
   protected LRUMap<ByteSequence,Boolean> cache;
   protected Authorizations authorizations;
@@ -53,7 +51,7 @@
   private VisibilityFilter(SortedKeyValueIterator<Key,Value> iterator,
       Authorizations authorizations, byte[] defaultVisibility) {
     super(iterator);
-    this.ve = new VisibilityEvaluator(authorizations);
+    this.ve = AccessEvaluator.of(authorizations.toAccessAuthorizations());
     this.authorizations = authorizations;
     this.defaultVisibility = new ArrayByteSequence(defaultVisibility);
     this.cache = new LRUMap<>(1000);
@@ -80,14 +78,11 @@
     }
 
     try {
-      boolean bb = ve.evaluate(new ColumnVisibility(testVis.toArray()));
+      boolean bb = ve.canAccess(testVis.toArray());
       cache.put(testVis, bb);
       return bb;
-    } catch (VisibilityParseException e) {
-      log.error("VisibilityParseException with visibility of Key: {}", k, e);
-      return false;
-    } catch (BadArgumentException e) {
-      log.error("BadArgumentException with visibility of Key: {}", k, e);
+    } catch (InvalidAccessExpressionException e) {
+      log.error("IllegalAccessExpressionException with visibility of Key: {}", k, e);
       return false;
     }
   }
diff --git a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ServiceLock.java b/core/src/main/java/org/apache/accumulo/core/lock/ServiceLock.java
similarity index 91%
rename from core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ServiceLock.java
rename to core/src/main/java/org/apache/accumulo/core/lock/ServiceLock.java
index 63807d0..4338805 100644
--- a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ServiceLock.java
+++ b/core/src/main/java/org/apache/accumulo/core/lock/ServiceLock.java
@@ -16,16 +16,20 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.accumulo.core.fate.zookeeper;
+package org.apache.accumulo.core.lock;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
 import static java.util.Objects.requireNonNull;
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Optional;
 import java.util.UUID;
 
+import org.apache.accumulo.core.fate.zookeeper.ZooCache;
 import org.apache.accumulo.core.fate.zookeeper.ZooCache.ZcStat;
+import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
 import org.apache.accumulo.core.fate.zookeeper.ZooUtil.LockID;
 import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeMissingPolicy;
 import org.apache.zookeeper.CreateMode;
@@ -114,9 +118,9 @@
       zooKeeper.exists(path.toString(), this);
       watchingParent = true;
       this.vmLockPrefix = new Prefix(ZLOCK_PREFIX + uuid.toString() + "#");
-    } catch (Exception ex) {
+    } catch (KeeperException | InterruptedException ex) {
       LOG.error("Error setting initial watch", ex);
-      throw new RuntimeException(ex);
+      throw new IllegalStateException(ex);
     }
   }
 
@@ -151,12 +155,12 @@
 
   }
 
-  public synchronized boolean tryLock(LockWatcher lw, byte[] data)
+  public synchronized boolean tryLock(LockWatcher lw, ServiceLockData lockData)
       throws KeeperException, InterruptedException {
 
     LockWatcherWrapper lww = new LockWatcherWrapper(lw);
 
-    lock(lww, data);
+    lock(lww, lockData);
 
     if (lww.acquiredLock) {
       return true;
@@ -284,7 +288,7 @@
     if (!children.contains(createdEphemeralNode)) {
       LOG.error("Expected ephemeral node {} to be in the list of children {}", createdEphemeralNode,
           children);
-      throw new RuntimeException(
+      throw new IllegalStateException(
           "Lock attempt ephemeral node no longer exist " + createdEphemeralNode);
     }
 
@@ -387,7 +391,7 @@
     localLw.lostLock(reason);
   }
 
-  public synchronized void lock(final AccumuloLockWatcher lw, byte[] data) {
+  public synchronized void lock(final AccumuloLockWatcher lw, ServiceLockData lockData) {
 
     if (lockWatcher != null || lockNodeName != null || createdNodeName != null) {
       throw new IllegalStateException();
@@ -401,9 +405,9 @@
       // except that instead of the ephemeral lock node being of the form guid-lock- use lock-guid-.
       // Another deviation from the recipe is that we cleanup any extraneous ephemeral nodes that
       // were created.
-      final String createPath =
-          zooKeeper.create(lockPathPrefix, data, ZooUtil.PUBLIC, CreateMode.EPHEMERAL_SEQUENTIAL);
-      LOG.debug("[{}] Ephemeral node {} created", vmLockPrefix, createPath);
+      final String createPath = zooKeeper.create(lockPathPrefix, lockData.serialize(),
+          ZooUtil.PUBLIC, CreateMode.EPHEMERAL_SEQUENTIAL);
+      LOG.debug("[{}] Ephemeral node {} created with data: {}", vmLockPrefix, createPath, lockData);
 
       // It's possible that the call above was retried several times and multiple ephemeral nodes
       // were created but the client missed the response for some reason. Find the ephemeral nodes
@@ -412,7 +416,8 @@
       if (!children.contains(createPath.substring(path.toString().length() + 1))) {
         LOG.error("Expected ephemeral node {} to be in the list of children {}", createPath,
             children);
-        throw new RuntimeException("Lock attempt ephemeral node no longer exist " + createPath);
+        throw new IllegalStateException(
+            "Lock attempt ephemeral node no longer exist " + createPath);
       }
 
       String lowestSequentialPath = null;
@@ -446,9 +451,6 @@
           }
         }
       }
-      if (lowestSequentialPath == null) {
-        throw new IllegalStateException("Could not find lowest sequential path under " + path);
-      }
       final String pathForWatcher = lowestSequentialPath;
 
       // Set a watcher on the lowest sequential node that we created, this handles the case
@@ -598,9 +600,12 @@
     return lockNodeName != null;
   }
 
-  public synchronized void replaceLockData(byte[] b) throws KeeperException, InterruptedException {
+  public synchronized void replaceLockData(ServiceLockData lockData)
+      throws KeeperException, InterruptedException {
     if (getLockPath() != null) {
-      zooKeeper.setData(getLockPath(), b, -1);
+      zooKeeper.setData(getLockPath(), lockData.serialize(), -1);
+      LOG.debug("[{}] Lock data replaced at path {} with data: {}", vmLockPrefix, getLockPath(),
+          lockData);
     }
   }
 
@@ -652,36 +657,44 @@
     return zc.get(lid.path + "/" + lid.node, stat) != null && stat.getEphemeralOwner() == lid.eid;
   }
 
-  public static byte[] getLockData(ZooKeeper zk, ServiceLockPath path)
+  public static Optional<ServiceLockData> getLockData(ZooKeeper zk, ServiceLockPath path)
       throws KeeperException, InterruptedException {
 
     List<String> children = validateAndSort(path, zk.getChildren(path.toString(), null));
 
     if (children.isEmpty()) {
-      return null;
+      return Optional.empty();
     }
 
     String lockNode = children.get(0);
 
-    return zk.getData(path + "/" + lockNode, false, null);
+    byte[] data = zk.getData(path + "/" + lockNode, false, null);
+    if (data == null) {
+      data = new byte[0];
+    }
+    return ServiceLockData.parse(data);
   }
 
-  public static byte[] getLockData(org.apache.accumulo.core.fate.zookeeper.ZooCache zc,
-      ServiceLockPath path, ZcStat stat) {
+  public static Optional<ServiceLockData> getLockData(
+      org.apache.accumulo.core.fate.zookeeper.ZooCache zc, ServiceLockPath path, ZcStat stat) {
 
     List<String> children = validateAndSort(path, zc.getChildren(path.toString()));
 
     if (children.isEmpty()) {
-      return null;
+      return Optional.empty();
     }
 
     String lockNode = children.get(0);
 
     if (!lockNode.startsWith(ZLOCK_PREFIX)) {
-      throw new RuntimeException("Node " + lockNode + " at " + path + " is not a lock node");
+      throw new IllegalStateException("Node " + lockNode + " at " + path + " is not a lock node");
     }
 
-    return zc.get(path + "/" + lockNode, stat);
+    byte[] data = zc.get(path + "/" + lockNode, stat);
+    if (data == null) {
+      data = new byte[0];
+    }
+    return ServiceLockData.parse(data);
   }
 
   public static long getSessionId(ZooCache zc, ServiceLockPath path) {
@@ -727,7 +740,7 @@
     String lockNode = children.get(0);
 
     if (!lockNode.startsWith(ZLOCK_PREFIX)) {
-      throw new RuntimeException("Node " + lockNode + " at " + path + " is not a lock node");
+      throw new IllegalStateException("Node " + lockNode + " at " + path + " is not a lock node");
     }
 
     String pathToDelete = path + "/" + lockNode;
@@ -748,7 +761,7 @@
     String lockNode = children.get(0);
 
     if (!lockNode.startsWith(ZLOCK_PREFIX)) {
-      throw new RuntimeException("Node " + lockNode + " at " + path + " is not a lock node");
+      throw new IllegalStateException("Node " + lockNode + " at " + path + " is not a lock node");
     }
 
     byte[] data = zk.getData(path + "/" + lockNode);
diff --git a/core/src/main/java/org/apache/accumulo/core/lock/ServiceLockData.java b/core/src/main/java/org/apache/accumulo/core/lock/ServiceLockData.java
new file mode 100644
index 0000000..c550d64
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/lock/ServiceLockData.java
@@ -0,0 +1,221 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.lock;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static java.util.Objects.requireNonNull;
+import static org.apache.accumulo.core.util.LazySingletons.GSON;
+
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.HashSet;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+
+import org.apache.accumulo.core.util.AddressUtil;
+
+import com.google.common.net.HostAndPort;
+
+public class ServiceLockData implements Comparable<ServiceLockData> {
+
+  /**
+   * Thrift Service list
+   */
+  public static enum ThriftService {
+    CLIENT,
+    COORDINATOR,
+    COMPACTOR,
+    FATE,
+    GC,
+    MANAGER,
+    NONE,
+    TABLET_INGEST,
+    TABLET_MANAGEMENT,
+    TABLET_SCAN,
+    TSERV
+  }
+
+  /**
+   * An object that describes a process, the group assigned to that process, the Thrift service and
+   * the address to use to communicate with that service.
+   */
+  public static class ServiceDescriptor {
+
+    /**
+     * The group name that will be used when one is not specified.
+     */
+    public static final String DEFAULT_GROUP_NAME = "default";
+
+    private final UUID uuid;
+    private final ThriftService service;
+    private final String address;
+    private final String group;
+
+    public ServiceDescriptor(UUID uuid, ThriftService service, String address) {
+      this(uuid, service, address, DEFAULT_GROUP_NAME);
+    }
+
+    public ServiceDescriptor(UUID uuid, ThriftService service, String address, String group) {
+      this.uuid = requireNonNull(uuid);
+      this.service = requireNonNull(service);
+      this.address = requireNonNull(address);
+      this.group = requireNonNull(group);
+    }
+
+    public UUID getUUID() {
+      return uuid;
+    }
+
+    public ThriftService getService() {
+      return service;
+    }
+
+    public String getAddress() {
+      return address;
+    }
+
+    public String getGroup() {
+      return group;
+    }
+
+    @Override
+    public int hashCode() {
+      return toString().hashCode();
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (this == obj) {
+        return true;
+      }
+      if (obj == null) {
+        return false;
+      }
+      if (getClass() != obj.getClass()) {
+        return false;
+      }
+      ServiceDescriptor other = (ServiceDescriptor) obj;
+      return toString().equals(other.toString());
+    }
+
+    @Override
+    public String toString() {
+      return GSON.get().toJson(this);
+    }
+
+  }
+
+  /**
+   * A set of ServiceDescriptor's
+   */
+  public static class ServiceDescriptors {
+    private final Set<ServiceDescriptor> descriptors;
+
+    public ServiceDescriptors() {
+      descriptors = new HashSet<>();
+    }
+
+    public ServiceDescriptors(HashSet<ServiceDescriptor> descriptors) {
+      this.descriptors = descriptors;
+    }
+
+    public void addService(ServiceDescriptor sd) {
+      this.descriptors.add(sd);
+    }
+
+    public Set<ServiceDescriptor> getServices() {
+      return descriptors;
+    }
+  }
+
+  private EnumMap<ThriftService,ServiceDescriptor> services;
+
+  public ServiceLockData(ServiceDescriptors sds) {
+    this.services = new EnumMap<>(ThriftService.class);
+    sds.getServices().forEach(sd -> this.services.put(sd.getService(), sd));
+  }
+
+  public ServiceLockData(UUID uuid, String address, ThriftService service, String group) {
+    this(new ServiceDescriptors(new HashSet<>(
+        Collections.singleton(new ServiceDescriptor(uuid, service, address, group)))));
+  }
+
+  public ServiceLockData(UUID uuid, String address, ThriftService service) {
+    this(new ServiceDescriptors(
+        new HashSet<>(Collections.singleton(new ServiceDescriptor(uuid, service, address)))));
+  }
+
+  public String getAddressString(ThriftService service) {
+    ServiceDescriptor sd = services.get(service);
+    return sd == null ? null : sd.getAddress();
+  }
+
+  public HostAndPort getAddress(ThriftService service) {
+    String s = getAddressString(service);
+    return s == null ? null : AddressUtil.parseAddress(s);
+  }
+
+  public String getGroup(ThriftService service) {
+    ServiceDescriptor sd = services.get(service);
+    return sd == null ? null : sd.getGroup();
+  }
+
+  public UUID getServerUUID(ThriftService service) {
+    ServiceDescriptor sd = services.get(service);
+    return sd == null ? null : sd.getUUID();
+  }
+
+  public byte[] serialize() {
+    ServiceDescriptors sd = new ServiceDescriptors();
+    services.values().forEach(s -> sd.addService(s));
+    return GSON.get().toJson(sd).getBytes(UTF_8);
+  }
+
+  @Override
+  public String toString() {
+    return new String(serialize(), UTF_8);
+  }
+
+  @Override
+  public int hashCode() {
+    return toString().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    return o instanceof ServiceLockData ? Objects.equals(toString(), o.toString()) : false;
+  }
+
+  @Override
+  public int compareTo(ServiceLockData other) {
+    return toString().compareTo(other.toString());
+  }
+
+  public static Optional<ServiceLockData> parse(byte[] lockData) {
+    if (lockData == null) {
+      return Optional.empty();
+    }
+    String data = new String(lockData, UTF_8);
+    return data.isBlank() ? Optional.empty()
+        : Optional.of(new ServiceLockData(GSON.get().fromJson(data, ServiceDescriptors.class)));
+  }
+
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/logging/FateLogger.java b/core/src/main/java/org/apache/accumulo/core/logging/FateLogger.java
index ccad01a..fa7ed86 100644
--- a/core/src/main/java/org/apache/accumulo/core/logging/FateLogger.java
+++ b/core/src/main/java/org/apache/accumulo/core/logging/FateLogger.java
@@ -21,9 +21,9 @@
 import static org.apache.accumulo.core.fate.FateTxId.formatTid;
 
 import java.io.Serializable;
+import java.time.Duration;
 import java.util.EnumSet;
 import java.util.List;
-import java.util.concurrent.TimeUnit;
 import java.util.function.Function;
 
 import org.apache.accumulo.core.fate.Fate;
@@ -62,8 +62,8 @@
       }
 
       @Override
-      public void unreserve(long tid, long deferTime, TimeUnit deferTimeUnit) {
-        store.unreserve(tid, deferTime, deferTimeUnit);
+      public void unreserve(long tid, Duration deferTime) {
+        store.unreserve(tid, deferTime);
       }
 
       @Override
diff --git a/core/src/main/java/org/apache/accumulo/core/logging/TabletLogger.java b/core/src/main/java/org/apache/accumulo/core/logging/TabletLogger.java
index c91eb9f..2209e41 100644
--- a/core/src/main/java/org/apache/accumulo/core/logging/TabletLogger.java
+++ b/core/src/main/java/org/apache/accumulo/core/logging/TabletLogger.java
@@ -23,23 +23,26 @@
 import java.util.Collection;
 import java.util.List;
 import java.util.Optional;
-import java.util.concurrent.TimeUnit;
+import java.util.UUID;
 
 import org.apache.accumulo.core.client.admin.CompactionConfig;
 import org.apache.accumulo.core.client.admin.compaction.CompactableFile;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.metadata.CompactableFileImpl;
 import org.apache.accumulo.core.metadata.StoredTabletFile;
 import org.apache.accumulo.core.metadata.TServerInstance;
 import org.apache.accumulo.core.metadata.TabletFile;
+import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
 import org.apache.accumulo.core.spi.compaction.CompactionJob;
 import org.apache.accumulo.core.spi.compaction.CompactionKind;
 import org.apache.accumulo.core.tabletserver.log.LogEntry;
-import org.apache.accumulo.core.util.HostAndPort;
+import org.apache.accumulo.core.util.time.SteadyTime;
 import org.apache.commons.io.FileUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.common.collect.Collections2;
+import com.google.common.net.HostAndPort;
 
 /**
  * This class contains source for logs messages about a tablets internal state, like its location,
@@ -72,10 +75,9 @@
     locLog.debug("Loading {} on {}", extent, server);
   }
 
-  public static void suspended(KeyExtent extent, HostAndPort server, long time, TimeUnit timeUnit,
+  public static void suspended(KeyExtent extent, HostAndPort server, SteadyTime time,
       int numWalogs) {
-    locLog.debug("Suspended {} to {} at {} ms with {} walogs", extent, server,
-        timeUnit.toMillis(time), numWalogs);
+    locLog.debug("Suspended {} to {} at {} ms with {} walogs", extent, server, time, numWalogs);
   }
 
   public static void unsuspended(KeyExtent extent) {
@@ -117,32 +119,45 @@
    * Lazily converts TableFile to file names. The lazy part is really important because when it is
    * not called with log.isDebugEnabled().
    */
-  private static Collection<String> asFileNames(Collection<CompactableFile> files) {
-    return Collections2.transform(files, CompactableFile::getFileName);
+  private static Collection<String> asMinimalString(Collection<CompactableFile> files) {
+    return Collections2.transform(files,
+        cf -> CompactableFileImpl.toStoredTabletFile(cf).toMinimalString());
   }
 
   public static void selected(KeyExtent extent, CompactionKind kind,
-      Collection<? extends TabletFile> inputs) {
+      Collection<StoredTabletFile> inputs) {
     fileLog.trace("{} changed compaction selection set for {} new set {}", extent, kind,
-        Collections2.transform(inputs, TabletFile::getFileName));
+        Collections2.transform(inputs, StoredTabletFile::toMinimalString));
   }
 
   public static void compacting(KeyExtent extent, CompactionJob job, CompactionConfig config) {
     if (fileLog.isDebugEnabled()) {
       if (config == null) {
         fileLog.debug("Compacting {} on {} for {} from {} size {}", extent, job.getExecutor(),
-            job.getKind(), asFileNames(job.getFiles()), getSize(job.getFiles()));
+            job.getKind(), asMinimalString(job.getFiles()), getSize(job.getFiles()));
       } else {
         fileLog.debug("Compacting {} on {} for {} from {} size {} config {}", extent,
-            job.getExecutor(), job.getKind(), asFileNames(job.getFiles()), getSize(job.getFiles()),
-            config);
+            job.getExecutor(), job.getKind(), asMinimalString(job.getFiles()),
+            getSize(job.getFiles()), config);
       }
     }
   }
 
-  public static void compacted(KeyExtent extent, CompactionJob job, TabletFile output) {
+  public static void compacted(KeyExtent extent, CompactionJob job, StoredTabletFile output) {
     fileLog.debug("Compacted {} for {} created {} from {}", extent, job.getKind(), output,
-        asFileNames(job.getFiles()));
+        asMinimalString(job.getFiles()));
+  }
+
+  public static void compactionFailed(KeyExtent extent, CompactionJob job,
+      CompactionConfig config) {
+    fileLog.debug("Failed to compact: extent: {}, input files: {}, iterators: {}", extent,
+        asMinimalString(job.getFiles()), config.getIterators());
+  }
+
+  public static void externalCompactionFailed(KeyExtent extent, ExternalCompactionId id,
+      CompactionJob job, CompactionConfig config) {
+    fileLog.debug("Failed to compact: id: {}, extent: {}, input files: {}, iterators: {}", id,
+        extent, asMinimalString(job.getFiles()), config.getIterators());
   }
 
   public static void flushed(KeyExtent extent, Optional<StoredTabletFile> newDatafile) {
@@ -159,7 +174,7 @@
 
   public static void recovering(KeyExtent extent, List<LogEntry> logEntries) {
     if (recoveryLog.isDebugEnabled()) {
-      List<String> logIds = logEntries.stream().map(LogEntry::getUniqueID).collect(toList());
+      List<UUID> logIds = logEntries.stream().map(LogEntry::getUniqueID).collect(toList());
       recoveryLog.debug("For {} recovering data from walogs: {}", extent, logIds);
     }
   }
diff --git a/core/src/main/java/org/apache/accumulo/core/manager/balancer/AssignmentParamsImpl.java b/core/src/main/java/org/apache/accumulo/core/manager/balancer/AssignmentParamsImpl.java
index ca7b6e7..14ccaa5 100644
--- a/core/src/main/java/org/apache/accumulo/core/manager/balancer/AssignmentParamsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/manager/balancer/AssignmentParamsImpl.java
@@ -27,7 +27,7 @@
 import org.apache.accumulo.core.data.TabletId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.dataImpl.TabletIdImpl;
-import org.apache.accumulo.core.master.thrift.TabletServerStatus;
+import org.apache.accumulo.core.manager.thrift.TabletServerStatus;
 import org.apache.accumulo.core.metadata.TServerInstance;
 import org.apache.accumulo.core.spi.balancer.TabletBalancer;
 import org.apache.accumulo.core.spi.balancer.data.TServerStatus;
diff --git a/core/src/main/java/org/apache/accumulo/core/manager/balancer/BalanceParamsImpl.java b/core/src/main/java/org/apache/accumulo/core/manager/balancer/BalanceParamsImpl.java
index a0c30d4..a7cc522 100644
--- a/core/src/main/java/org/apache/accumulo/core/manager/balancer/BalanceParamsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/manager/balancer/BalanceParamsImpl.java
@@ -27,7 +27,7 @@
 import org.apache.accumulo.core.data.TabletId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.dataImpl.TabletIdImpl;
-import org.apache.accumulo.core.master.thrift.TabletServerStatus;
+import org.apache.accumulo.core.manager.thrift.TabletServerStatus;
 import org.apache.accumulo.core.metadata.TServerInstance;
 import org.apache.accumulo.core.spi.balancer.TabletBalancer;
 import org.apache.accumulo.core.spi.balancer.data.TServerStatus;
diff --git a/core/src/main/java/org/apache/accumulo/core/manager/balancer/TServerStatusImpl.java b/core/src/main/java/org/apache/accumulo/core/manager/balancer/TServerStatusImpl.java
index d1a762d..2d6bf37 100644
--- a/core/src/main/java/org/apache/accumulo/core/manager/balancer/TServerStatusImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/manager/balancer/TServerStatusImpl.java
@@ -23,7 +23,7 @@
 import java.util.HashMap;
 import java.util.Map;
 
-import org.apache.accumulo.core.master.thrift.TabletServerStatus;
+import org.apache.accumulo.core.manager.thrift.TabletServerStatus;
 import org.apache.accumulo.core.spi.balancer.data.TServerStatus;
 import org.apache.accumulo.core.spi.balancer.data.TableStatistics;
 
diff --git a/core/src/main/java/org/apache/accumulo/core/manager/balancer/TableStatisticsImpl.java b/core/src/main/java/org/apache/accumulo/core/manager/balancer/TableStatisticsImpl.java
index 348152c..449b91d 100644
--- a/core/src/main/java/org/apache/accumulo/core/manager/balancer/TableStatisticsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/manager/balancer/TableStatisticsImpl.java
@@ -18,7 +18,7 @@
  */
 package org.apache.accumulo.core.manager.balancer;
 
-import org.apache.accumulo.core.master.thrift.TableInfo;
+import org.apache.accumulo.core.manager.thrift.TableInfo;
 import org.apache.accumulo.core.spi.balancer.data.TableStatistics;
 
 public class TableStatisticsImpl implements TableStatistics {
diff --git a/core/src/main/java/org/apache/accumulo/core/manager/balancer/TabletServerIdImpl.java b/core/src/main/java/org/apache/accumulo/core/manager/balancer/TabletServerIdImpl.java
index 9eff55a..2fefb46 100644
--- a/core/src/main/java/org/apache/accumulo/core/manager/balancer/TabletServerIdImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/manager/balancer/TabletServerIdImpl.java
@@ -22,7 +22,8 @@
 
 import org.apache.accumulo.core.metadata.TServerInstance;
 import org.apache.accumulo.core.spi.balancer.data.TabletServerId;
-import org.apache.accumulo.core.util.HostAndPort;
+
+import com.google.common.net.HostAndPort;
 
 /**
  * @since 2.1.0
diff --git a/core/src/main/java/org/apache/accumulo/core/manager/balancer/TabletStatisticsImpl.java b/core/src/main/java/org/apache/accumulo/core/manager/balancer/TabletStatisticsImpl.java
index 4795489..5d5977b 100644
--- a/core/src/main/java/org/apache/accumulo/core/manager/balancer/TabletStatisticsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/manager/balancer/TabletStatisticsImpl.java
@@ -46,6 +46,7 @@
   }
 
   @Override
+  @Deprecated
   public long getSplitCreationTime() {
     return thriftStats.getSplitCreationTime();
   }
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/AbstractTabletFile.java b/core/src/main/java/org/apache/accumulo/core/metadata/AbstractTabletFile.java
new file mode 100644
index 0000000..aff3ee2
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/AbstractTabletFile.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.metadata;
+
+import static org.apache.accumulo.core.util.RowRangeUtil.requireKeyExtentDataRange;
+import static org.apache.accumulo.core.util.RowRangeUtil.stripZeroTail;
+
+import java.util.Objects;
+
+import org.apache.accumulo.core.data.Range;
+import org.apache.hadoop.fs.Path;
+
+/**
+ * A base class used to represent file references that are handled by code that processes tablet
+ * files.
+ *
+ * @since 3.0.0
+ */
+public abstract class AbstractTabletFile<T extends AbstractTabletFile<T>>
+    implements TabletFile, Comparable<T> {
+
+  protected final Path path;
+  protected final Range range;
+
+  protected AbstractTabletFile(Path path, Range range) {
+    this.path = Objects.requireNonNull(path);
+    this.range = requireKeyExtentDataRange(range);
+  }
+
+  @Override
+  public Path getPath() {
+    return path;
+  }
+
+  @Override
+  public Range getRange() {
+    return range;
+  }
+
+  @Override
+  public boolean hasRange() {
+    return !range.isInfiniteStartKey() || !range.isInfiniteStopKey();
+  }
+
+  @Override
+  public String toMinimalString() {
+    if (hasRange()) {
+      String startRow = range.isInfiniteStartKey() ? "-inf"
+          : stripZeroTail(range.getStartKey().getRowData()).toString();
+      String endRow = range.isInfiniteStopKey() ? "+inf"
+          : stripZeroTail(range.getEndKey().getRowData()).toString();
+      return getFileName() + " (" + startRow + "," + endRow + "]";
+    } else {
+      return getFileName();
+    }
+  }
+
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/AccumuloTable.java b/core/src/main/java/org/apache/accumulo/core/metadata/AccumuloTable.java
new file mode 100644
index 0000000..568959d
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/AccumuloTable.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.metadata;
+
+import java.util.Arrays;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.accumulo.core.clientImpl.Namespace;
+import org.apache.accumulo.core.data.TableId;
+
+/**
+ * Defines the name and id of all tables in the accumulo table namespace.
+ */
+public enum AccumuloTable {
+
+  ROOT("root", "+r"), METADATA("metadata", "!0"), SCAN_REF("scanref", "+scanref");
+
+  private final String name;
+  private final TableId tableId;
+
+  public String tableName() {
+    return name;
+  }
+
+  public TableId tableId() {
+    return tableId;
+  }
+
+  AccumuloTable(String name, String id) {
+    this.name = Namespace.ACCUMULO.name() + "." + name;
+    this.tableId = TableId.of(id);
+  }
+
+  private static final Set<TableId> ALL_IDS =
+      Arrays.stream(values()).map(AccumuloTable::tableId).collect(Collectors.toUnmodifiableSet());
+
+  public static Set<TableId> allTableIds() {
+    return ALL_IDS;
+  }
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/CompactableFileImpl.java b/core/src/main/java/org/apache/accumulo/core/metadata/CompactableFileImpl.java
index e8cf109..b25e83d 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/CompactableFileImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/CompactableFileImpl.java
@@ -22,6 +22,7 @@
 import java.util.Objects;
 
 import org.apache.accumulo.core.client.admin.compaction.CompactableFile;
+import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.metadata.schema.DataFileValue;
 
 public class CompactableFileImpl implements CompactableFile {
@@ -30,7 +31,12 @@
   private final DataFileValue dataFileValue;
 
   public CompactableFileImpl(URI uri, long size, long entries) {
-    this.storedTabletFile = new StoredTabletFile(uri.toString());
+    this.storedTabletFile = StoredTabletFile.of(uri);
+    this.dataFileValue = new DataFileValue(size, entries);
+  }
+
+  public CompactableFileImpl(URI uri, Range range, long size, long entries) {
+    this.storedTabletFile = StoredTabletFile.of(uri, range);
     this.dataFileValue = new DataFileValue(size, entries);
   }
 
@@ -45,6 +51,11 @@
   }
 
   @Override
+  public Range getRange() {
+    return storedTabletFile.getRange();
+  }
+
+  @Override
   public String getFileName() {
     return storedTabletFile.getFileName();
   }
@@ -89,6 +100,6 @@
 
   @Override
   public String toString() {
-    return "[" + storedTabletFile.getFileName() + ", " + dataFileValue + "]";
+    return "[" + storedTabletFile.toMinimalString() + ", " + dataFileValue + "]";
   }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/MetadataLocationObtainer.java b/core/src/main/java/org/apache/accumulo/core/metadata/MetadataLocationObtainer.java
index 9ac2658..80126d1 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/MetadataLocationObtainer.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/MetadataLocationObtainer.java
@@ -21,6 +21,7 @@
 import static java.util.concurrent.TimeUnit.SECONDS;
 
 import java.io.IOException;
+import java.io.UncheckedIOException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
@@ -88,8 +89,8 @@
 
       if (log.isTraceEnabled()) {
         log.trace("tid={} Looking up in {} row={} extent={} tserver={}",
-            Thread.currentThread().getId(), src.tablet_extent.tableId(), TextUtil.truncate(row),
-            src.tablet_extent, src.tablet_location);
+            Thread.currentThread().getId(), src.getExtent().tableId(), TextUtil.truncate(row),
+            src.getExtent(), src.getTserverLocation());
         timer = new OpTimer().start();
       }
 
@@ -105,8 +106,8 @@
       List<IterInfo> serverSideIteratorList = new ArrayList<>();
       serverSideIteratorList.add(new IterInfo(10000, WholeRowIterator.class.getName(), "WRI"));
       Map<String,Map<String,String>> serverSideIteratorOptions = Collections.emptyMap();
-      boolean more = ThriftScanner.getBatchFromServer(context, range, src.tablet_extent,
-          src.tablet_location, encodedResults, locCols, serverSideIteratorList,
+      boolean more = ThriftScanner.getBatchFromServer(context, range, src.getExtent(),
+          src.getTserverLocation(), encodedResults, locCols, serverSideIteratorList,
           serverSideIteratorOptions, Constants.SCAN_BATCH_SIZE, Authorizations.EMPTY, 0L, null);
 
       decodeRows(encodedResults, results);
@@ -115,7 +116,7 @@
         range = new Range(results.lastKey().followingKey(PartialKey.ROW_COLFAM_COLQUAL_COLVIS_TIME),
             true, new Key(stopRow).followingKey(PartialKey.ROW), false);
         encodedResults.clear();
-        ThriftScanner.getBatchFromServer(context, range, src.tablet_extent, src.tablet_location,
+        ThriftScanner.getBatchFromServer(context, range, src.getExtent(), src.getTserverLocation(),
             encodedResults, locCols, serverSideIteratorList, serverSideIteratorOptions,
             Constants.SCAN_BATCH_SIZE, Authorizations.EMPTY, 0L, null);
 
@@ -125,7 +126,7 @@
       if (timer != null) {
         timer.stop();
         log.trace("tid={} Got {} results from {} in {}", Thread.currentThread().getId(),
-            results.size(), src.tablet_extent, String.format("%.3f secs", timer.scale(SECONDS)));
+            results.size(), src.getExtent(), String.format("%.3f secs", timer.scale(SECONDS)));
       }
 
       // if (log.isTraceEnabled()) log.trace("results "+results);
@@ -134,15 +135,15 @@
 
     } catch (AccumuloServerException ase) {
       if (log.isTraceEnabled()) {
-        log.trace("{} lookup failed, {} server side exception", src.tablet_extent.tableId(),
-            src.tablet_location);
+        log.trace("{} lookup failed, {} server side exception", src.getExtent().tableId(),
+            src.getTserverLocation());
       }
       throw ase;
     } catch (AccumuloException e) {
       if (log.isTraceEnabled()) {
-        log.trace("{} lookup failed", src.tablet_extent.tableId(), e);
+        log.trace("{} lookup failed", src.getExtent().tableId(), e);
       }
-      parent.invalidateCache(context, src.tablet_location);
+      parent.invalidateCache(context, src.getTserverLocation());
     }
 
     return null;
@@ -180,7 +181,7 @@
         try {
           results.putAll(WholeRowIterator.decodeRow(entry.getKey(), entry.getValue()));
         } catch (IOException e) {
-          throw new RuntimeException(e);
+          throw new UncheckedIOException(e);
         }
       }
     };
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/MetadataServicer.java b/core/src/main/java/org/apache/accumulo/core/metadata/MetadataServicer.java
index a4f6d2a..fc4e23f 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/MetadataServicer.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/MetadataServicer.java
@@ -42,9 +42,9 @@
 
   public static MetadataServicer forTableId(ClientContext context, TableId tableId) {
     checkArgument(tableId != null, "tableId is null");
-    if (RootTable.ID.equals(tableId)) {
+    if (AccumuloTable.ROOT.tableId().equals(tableId)) {
       return new ServicerForRootTable(context);
-    } else if (MetadataTable.ID.equals(tableId)) {
+    } else if (AccumuloTable.METADATA.tableId().equals(tableId)) {
       return new ServicerForMetadataTable(context);
     } else {
       return new ServicerForUserTables(context, tableId);
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/MetadataTable.java b/core/src/main/java/org/apache/accumulo/core/metadata/MetadataTable.java
deleted file mode 100644
index b2a9a5d..0000000
--- a/core/src/main/java/org/apache/accumulo/core/metadata/MetadataTable.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.metadata;
-
-import org.apache.accumulo.core.clientImpl.Namespace;
-import org.apache.accumulo.core.data.TableId;
-
-public class MetadataTable {
-  public static final TableId ID = TableId.of("!0");
-  public static final String NAME = Namespace.ACCUMULO.name() + ".metadata";
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/ReferencedTabletFile.java b/core/src/main/java/org/apache/accumulo/core/metadata/ReferencedTabletFile.java
new file mode 100644
index 0000000..4e219d8
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/ReferencedTabletFile.java
@@ -0,0 +1,238 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.metadata;
+
+import static org.apache.accumulo.core.Constants.HDFS_TABLES_DIR;
+
+import java.net.URI;
+import java.util.Comparator;
+import java.util.Objects;
+
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.fs.Path;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Object representing a tablet file that may exist in the metadata table. This class is used for
+ * reading and opening tablet files. It is also used when inserting new tablet files. When a new
+ * file is inserted, the {@link #insert()} method is called and returns a {@link StoredTabletFile}
+ * For situations where a tablet file needs to be updated or deleted in the metadata, a
+ * {@link StoredTabletFile} is required.
+ * <p>
+ * As of 2.1, Tablet file paths should now be only absolute URIs with the removal of relative paths
+ * in Upgrader9to10.upgradeRelativePaths()
+ */
+public class ReferencedTabletFile extends AbstractTabletFile<ReferencedTabletFile> {
+
+  public static class FileParts {
+
+    // parts of an absolute URI, like "hdfs://1.2.3.4/accumulo/tables/2a/t-0003/C0004.rf"
+    // volume: hdfs://1.2.3.4/accumulo
+    // tableId: 2a
+    // tabletDir: t-0003
+    // fileName: C0004.rf
+    // normalizedPath: hdfs://1.2.3.4/accumulo/tables/2a/t-0003/C0004.rf
+    private final String volume;
+    private final TableId tableId;
+    private final String tabletDir;
+    private final String fileName;
+    private final String normalizedPath;
+
+    public FileParts(String volume, TableId tableId, String tabletDir, String fileName,
+        String normalizedPath) {
+      this.volume = volume;
+      this.tableId = tableId;
+      this.tabletDir = tabletDir;
+      this.fileName = fileName;
+      this.normalizedPath = normalizedPath;
+    }
+
+    public String getVolume() {
+      return volume;
+    }
+
+    public TableId getTableId() {
+      return tableId;
+    }
+
+    public String getTabletDir() {
+      return tabletDir;
+    }
+
+    public String getFileName() {
+      return fileName;
+    }
+
+    public String getNormalizedPath() {
+      return normalizedPath;
+    }
+
+  }
+
+  private static String constructErrorMsg(Path filePath) {
+    return "Missing or invalid part of tablet file metadata entry: " + filePath;
+  }
+
+  public static FileParts parsePath(Path filePath) {
+    // File name construct: <volume>/<tablePath>/<tableId>/<tablet>/<file>
+    // Example: hdfs://namenode:9020/accumulo/tables/1/default_tablet/F00001.rf
+    final URI uri = filePath.toUri();
+
+    // validate that this is a fully qualified uri
+    Preconditions.checkArgument(uri.getScheme() != null, constructErrorMsg(filePath));
+
+    final String path = uri.getPath(); // ex: /accumulo/tables/1/default_tablet/F00001.rf
+    final String[] parts = path.split("/");
+    final int numParts = parts.length; // should contain tables, 1, default_tablet, F00001.rf
+
+    if (numParts < 4) {
+      throw new IllegalArgumentException(constructErrorMsg(filePath));
+    }
+
+    final String fileName = parts[numParts - 1];
+    final String tabletDirectory = parts[numParts - 2];
+    final TableId tableId = TableId.of(parts[numParts - 3]);
+    final String tablesPath = parts[numParts - 4];
+
+    // determine where file path starts, the rest is the volume
+    final String computedFilePath =
+        HDFS_TABLES_DIR + "/" + tableId.canonical() + "/" + tabletDirectory + "/" + fileName;
+    final String uriString = uri.toString();
+    int idx = uriString.lastIndexOf(computedFilePath);
+
+    if (idx == -1) {
+      throw new IllegalArgumentException(constructErrorMsg(filePath));
+    }
+
+    // The volume is the beginning portion of the uri up to the start
+    // of the file path.
+    final String volume = uriString.substring(0, idx);
+
+    if (StringUtils.isBlank(fileName) || StringUtils.isBlank(tabletDirectory)
+        || StringUtils.isBlank(tablesPath) || StringUtils.isBlank(volume)) {
+      throw new IllegalArgumentException(constructErrorMsg(filePath));
+    }
+    ValidationUtil.validateFileName(fileName);
+    Preconditions.checkArgument(tablesPath.equals(HDFS_TABLES_DIR_NAME),
+        "tables directory name is not " + HDFS_TABLES_DIR_NAME + ", is " + tablesPath);
+
+    final String normalizedPath = volume + computedFilePath;
+
+    if (!normalizedPath.equals(uriString)) {
+      throw new RuntimeException("Error parsing file path, " + normalizedPath + " != " + uriString);
+    }
+
+    return new FileParts(volume, tableId, tabletDirectory, fileName, normalizedPath);
+
+  }
+
+  private final FileParts parts;
+
+  private static final Logger log = LoggerFactory.getLogger(ReferencedTabletFile.class);
+  private static final String HDFS_TABLES_DIR_NAME = HDFS_TABLES_DIR.substring(1);
+
+  private static final Comparator<ReferencedTabletFile> comparator =
+      Comparator.comparing(ReferencedTabletFile::getNormalizedPathStr)
+          .thenComparing(ReferencedTabletFile::getRange);
+
+  public ReferencedTabletFile(Path metaPath) {
+    this(metaPath, new Range());
+  }
+
+  /**
+   * Construct new tablet file using a Path. Used in the case where we had to use Path object to
+   * qualify an absolute path or create a new file.
+   */
+  public ReferencedTabletFile(Path metaPath, Range range) {
+    super(Objects.requireNonNull(metaPath), range);
+    log.trace("Parsing TabletFile from {}", metaPath);
+    parts = parsePath(metaPath);
+  }
+
+  public String getVolume() {
+    return parts.getVolume();
+  }
+
+  public TableId getTableId() {
+    return parts.getTableId();
+  }
+
+  public String getTabletDir() {
+    return parts.getTabletDir();
+  }
+
+  @Override
+  public String getFileName() {
+    return parts.getFileName();
+  }
+
+  /**
+   * Return a string for opening and reading the tablet file. Doesn't have to be exact string in
+   * metadata.
+   */
+  public String getNormalizedPathStr() {
+    return parts.getNormalizedPath();
+  }
+
+  /**
+   * New file was written to metadata so return a StoredTabletFile
+   */
+  public StoredTabletFile insert() {
+    return StoredTabletFile.of(getPath(), getRange());
+  }
+
+  @Override
+  public int compareTo(ReferencedTabletFile o) {
+    return comparator.compare(this, o);
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj instanceof ReferencedTabletFile) {
+      ReferencedTabletFile that = (ReferencedTabletFile) obj;
+      return parts.getNormalizedPath().equals(that.parts.getNormalizedPath())
+          && range.equals(that.range);
+    }
+    return false;
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(parts.getNormalizedPath(), range);
+  }
+
+  @Override
+  public String toString() {
+    return parts.getNormalizedPath();
+  }
+
+  public static ReferencedTabletFile of(final Path path) {
+    return new ReferencedTabletFile(path);
+  }
+
+  public static ReferencedTabletFile of(final Path path, Range range) {
+    return new ReferencedTabletFile(path, range);
+  }
+
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/RootTable.java b/core/src/main/java/org/apache/accumulo/core/metadata/RootTable.java
index 7c00cc0..f7c5b0e 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/RootTable.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/RootTable.java
@@ -18,17 +18,11 @@
  */
 package org.apache.accumulo.core.metadata;
 
-import org.apache.accumulo.core.clientImpl.Namespace;
-import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
 
 public class RootTable {
 
-  public static final TableId ID = TableId.of("+r");
-
-  public static final String NAME = Namespace.ACCUMULO.name() + ".root";
-
   /**
    * DFS location relative to the Accumulo directory
    */
@@ -44,8 +38,8 @@
    */
   public static final String ZROOT_TABLET_GC_CANDIDATES = ZROOT_TABLET + "/gc_candidates";
 
-  public static final KeyExtent EXTENT = new KeyExtent(ID, null, null);
-  public static final KeyExtent OLD_EXTENT =
-      new KeyExtent(MetadataTable.ID, TabletsSection.encodeRow(MetadataTable.ID, null), null);
+  public static final KeyExtent EXTENT = new KeyExtent(AccumuloTable.ROOT.tableId(), null, null);
+  public static final KeyExtent OLD_EXTENT = new KeyExtent(AccumuloTable.METADATA.tableId(),
+      TabletsSection.encodeRow(AccumuloTable.METADATA.tableId(), null), null);
 
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/ScanServerRefStore.java b/core/src/main/java/org/apache/accumulo/core/metadata/ScanServerRefStore.java
new file mode 100644
index 0000000..461427f
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/ScanServerRefStore.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.metadata;
+
+import java.util.Collection;
+import java.util.UUID;
+import java.util.stream.Stream;
+
+public interface ScanServerRefStore {
+
+  /**
+   * Insert ScanServer references to Tablet files
+   *
+   * @param scanRefs set of scan server ref table file objects
+   */
+  default void put(Collection<ScanServerRefTabletFile> scanRefs) {
+    throw new UnsupportedOperationException();
+  }
+
+  /**
+   * Get ScanServer references to Tablet files
+   *
+   * @return stream of scan server references
+   */
+  default Stream<ScanServerRefTabletFile> list() {
+    throw new UnsupportedOperationException();
+  }
+
+  /**
+   * Delete the set of scan server references
+   *
+   * @param refsToDelete set of scan server references to delete
+   */
+  default void delete(Collection<ScanServerRefTabletFile> refsToDelete) {
+    throw new UnsupportedOperationException();
+  }
+
+  /**
+   * Delete scan server references for this server
+   *
+   * @param serverAddress address of server, cannot be null
+   * @param serverSessionId server session id, cannot be null
+   */
+  default void delete(String serverAddress, UUID serverSessionId) {
+    throw new UnsupportedOperationException();
+  }
+
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/ScanServerRefTabletFile.java b/core/src/main/java/org/apache/accumulo/core/metadata/ScanServerRefTabletFile.java
index f3365c7..50076ea 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/ScanServerRefTabletFile.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/ScanServerRefTabletFile.java
@@ -22,101 +22,38 @@
 import java.util.Objects;
 import java.util.UUID;
 
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.ScanServerFileReferenceSection;
-import org.apache.accumulo.core.util.UuidUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 
-import com.google.common.base.Preconditions;
+public class ScanServerRefTabletFile extends ReferencedTabletFile {
 
-public class ScanServerRefTabletFile extends TabletFile {
-
-  @SuppressWarnings("deprecation")
-  private static final String OLD_PREFIX =
-      org.apache.accumulo.core.metadata.schema.MetadataSchema.OldScanServerFileReferenceSection
-          .getRowPrefix();
-  private final String prefix;
   private final Value NULL_VALUE = new Value(new byte[0]);
-  private final Text serverAddress;
-  private final Text uuid;
+  private final Text colf;
+  private final Text colq;
 
-  public ScanServerRefTabletFile(UUID serverLockUUID, String serverAddress, String file) {
+  public ScanServerRefTabletFile(String file, String serverAddress, UUID serverLockUUID) {
     super(new Path(URI.create(file)));
-    // For new data, always use the current prefix
-    prefix = ScanServerFileReferenceSection.getRowPrefix();
-    this.serverAddress = new Text(serverAddress);
-    uuid = new Text(serverLockUUID.toString());
+    this.colf = new Text(serverAddress);
+    this.colq = new Text(serverLockUUID.toString());
   }
 
-  public ScanServerRefTabletFile(Key k) {
-    super(new Path(URI.create(extractFile(k))));
-    serverAddress = k.getColumnFamily();
-    if (isOldPrefix(k)) {
-      prefix = OLD_PREFIX;
-      uuid = new Text(k.getColumnQualifier().toString());
-    } else {
-      prefix = ScanServerFileReferenceSection.getRowPrefix();
-      var row = k.getRow().toString();
-      Preconditions.checkArgument(row.startsWith(prefix), "Unexpected row prefix %s ", row);
-      var uuidStr = row.substring(prefix.length());
-      Preconditions.checkArgument(UuidUtil.isUUID(uuidStr, 0), "Row suffix is not uuid %s", row);
-      uuid = new Text(uuidStr);
-    }
+  public ScanServerRefTabletFile(String file, Text colf, Text colq) {
+    super(new Path(URI.create(file)));
+    this.colf = colf;
+    this.colq = colq;
   }
 
-  public Mutation putMutation() {
-    // Only write scan refs in the new format
-    Mutation mutation = new Mutation(prefix + uuid.toString());
-    mutation.put(serverAddress, getFilePath(), getValue());
-    return mutation;
+  public String getRow() {
+    return this.getNormalizedPathStr();
   }
 
-  public Mutation putDeleteMutation() {
-    Mutation mutation;
-    if (Objects.equals(prefix, OLD_PREFIX)) {
-      mutation = new Mutation(prefix + this.getPath().toString());
-      mutation.putDelete(serverAddress, uuid);
-    } else {
-      mutation = new Mutation(prefix + uuid.toString());
-      mutation.putDelete(serverAddress, getFilePath());
-    }
-    return mutation;
+  public Text getServerAddress() {
+    return this.colf;
   }
 
-  private static String extractFile(Key k) {
-    if (isOldPrefix(k)) {
-      return k.getRow().toString().substring(OLD_PREFIX.length());
-    } else {
-      return k.getColumnQualifier().toString();
-    }
-  }
-
-  /**
-   * Returns the correctly formatted range for a unique uuid
-   *
-   * @param uuid ServerLockUUID of a Scan Server
-   * @return Range for a single scan server
-   */
-  public static Range getRange(UUID uuid) {
-    Objects.requireNonNull(uuid);
-    return new Range(MetadataSchema.ScanServerFileReferenceSection.getRowPrefix() + uuid);
-  }
-
-  private static boolean isOldPrefix(Key k) {
-    return k.getRow().toString().startsWith(OLD_PREFIX);
-  }
-
-  public UUID getServerLockUUID() {
-    return UUID.fromString(uuid.toString());
-  }
-
-  public Text getFilePath() {
-    return new Text(this.getPath().toString());
+  public Text getServerLockUUID() {
+    return this.colq;
   }
 
   public Value getValue() {
@@ -127,8 +64,8 @@
   public int hashCode() {
     final int prime = 31;
     int result = super.hashCode();
-    result = prime * result + ((serverAddress == null) ? 0 : serverAddress.hashCode());
-    result = prime * result + ((uuid == null) ? 0 : uuid.hashCode());
+    result = prime * result + ((colf == null) ? 0 : colf.hashCode());
+    result = prime * result + ((colq == null) ? 0 : colq.hashCode());
     return result;
   }
 
@@ -144,13 +81,13 @@
       return false;
     }
     ScanServerRefTabletFile other = (ScanServerRefTabletFile) obj;
-    return Objects.equals(serverAddress, other.serverAddress) && Objects.equals(uuid, other.uuid);
+    return Objects.equals(colf, other.colf) && Objects.equals(colq, other.colq);
   }
 
   @Override
   public String toString() {
-    return "ScanServerRefTabletFile [file=" + this.getPath().toString() + ", server address="
-        + serverAddress + ", server lock uuid=" + uuid + "]";
+    return "ScanServerRefTabletFile [file=" + this.getRow() + ", server address=" + colf
+        + ", server lock uuid=" + colq + "]";
   }
 
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/ServicerForMetadataTable.java b/core/src/main/java/org/apache/accumulo/core/metadata/ServicerForMetadataTable.java
index 948fcc1..a4b4150 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/ServicerForMetadataTable.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/ServicerForMetadataTable.java
@@ -27,7 +27,7 @@
 class ServicerForMetadataTable extends TableMetadataServicer {
 
   public ServicerForMetadataTable(ClientContext context) {
-    super(context, RootTable.NAME, MetadataTable.ID);
+    super(context, AccumuloTable.ROOT.tableName(), AccumuloTable.METADATA.tableId());
   }
 
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/ServicerForRootTable.java b/core/src/main/java/org/apache/accumulo/core/metadata/ServicerForRootTable.java
index aa85fc9..94e8644 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/ServicerForRootTable.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/ServicerForRootTable.java
@@ -38,7 +38,7 @@
 
   @Override
   public TableId getServicedTableId() {
-    return RootTable.ID;
+    return AccumuloTable.ROOT.tableId();
   }
 
   @Override
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/ServicerForUserTables.java b/core/src/main/java/org/apache/accumulo/core/metadata/ServicerForUserTables.java
index eeb7f7a..369bcc4 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/ServicerForUserTables.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/ServicerForUserTables.java
@@ -28,7 +28,7 @@
 class ServicerForUserTables extends TableMetadataServicer {
 
   public ServicerForUserTables(ClientContext context, TableId tableId) {
-    super(context, MetadataTable.NAME, tableId);
+    super(context, AccumuloTable.METADATA.tableName(), tableId);
   }
 
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/StoredTabletFile.java b/core/src/main/java/org/apache/accumulo/core/metadata/StoredTabletFile.java
index 24da313..0f66759 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/StoredTabletFile.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/StoredTabletFile.java
@@ -18,10 +18,27 @@
  */
 package org.apache.accumulo.core.metadata;
 
-import java.net.URI;
+import static org.apache.accumulo.core.util.RowRangeUtil.requireKeyExtentDataRange;
 
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.net.URI;
+import java.util.Comparator;
+import java.util.Objects;
+
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.util.json.ByteArrayToBase64TypeAdapter;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.Text;
+import org.checkerframework.checker.nullness.qual.NonNull;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.gson.Gson;
 
 /**
  * Object representing a tablet file entry stored in the metadata table. Keeps a string of the exact
@@ -34,16 +51,31 @@
  * As of 2.1, Tablet file paths should now be only absolute URIs with the removal of relative paths
  * in Upgrader9to10.upgradeRelativePaths()
  */
-public class StoredTabletFile extends TabletFile {
+public class StoredTabletFile extends AbstractTabletFile<StoredTabletFile> {
   private final String metadataEntry;
+  private final ReferencedTabletFile referencedTabletFile;
+  private final String metadataEntryPath;
+
+  private static final Comparator<StoredTabletFile> comparator = Comparator
+      .comparing(StoredTabletFile::getMetadataPath).thenComparing(StoredTabletFile::getRange);
 
   /**
    * Construct a tablet file using the string read from the metadata. Preserve the exact string so
    * the entry can be deleted.
    */
   public StoredTabletFile(String metadataEntry) {
-    super(new Path(URI.create(metadataEntry)));
-    this.metadataEntry = metadataEntry;
+    this(metadataEntry, deserialize(metadataEntry));
+  }
+
+  private StoredTabletFile(TabletFileCq fileCq) {
+    this(serialize(fileCq), fileCq);
+  }
+
+  private StoredTabletFile(String metadataEntry, TabletFileCq fileCq) {
+    super(Objects.requireNonNull(fileCq).path, fileCq.range);
+    this.metadataEntry = Objects.requireNonNull(metadataEntry);
+    this.metadataEntryPath = fileCq.path.toString();
+    this.referencedTabletFile = ReferencedTabletFile.of(getPath(), fileCq.range);
   }
 
   /**
@@ -51,15 +83,217 @@
    * and deleting metadata entries. If the exact string is not used, erroneous entries can pollute
    * the metadata table.
    */
-  public String getMetaUpdateDelete() {
+  public String getMetadata() {
     return metadataEntry;
   }
 
   /**
-   * Return a new Text object of {@link #getMetaUpdateDelete()}
+   * Returns just the Path portion of the metadata, not the full Json.
    */
-  public Text getMetaUpdateDeleteText() {
-    return new Text(getMetaUpdateDelete());
+  public String getMetadataPath() {
+    return metadataEntryPath;
   }
 
+  /**
+   * Return a new Text object of {@link #getMetadata()}
+   */
+  public Text getMetadataText() {
+    return new Text(getMetadata());
+  }
+
+  public ReferencedTabletFile getTabletFile() {
+    return referencedTabletFile;
+  }
+
+  public TableId getTableId() {
+    return referencedTabletFile.getTableId();
+  }
+
+  @Override
+  public String getFileName() {
+    return referencedTabletFile.getFileName();
+  }
+
+  public String getNormalizedPathStr() {
+    return referencedTabletFile.getNormalizedPathStr();
+  }
+
+  @Override
+  public int compareTo(StoredTabletFile o) {
+    if (equals(o)) {
+      return 0;
+    } else {
+      return comparator.compare(this, o);
+    }
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    StoredTabletFile that = (StoredTabletFile) o;
+    return Objects.equals(metadataEntry, that.metadataEntry);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(metadataEntry);
+  }
+
+  @Override
+  public String toString() {
+    return metadataEntry;
+  }
+
+  /**
+   * Validates that the provided metadata string for the StoredTabletFile is valid.
+   */
+  public static void validate(String metadataEntry) {
+    final TabletFileCq tabletFileCq = deserialize(metadataEntry);
+    // Validate the path
+    ReferencedTabletFile.parsePath(deserialize(metadataEntry).path);
+    // Validate the range
+    requireKeyExtentDataRange(tabletFileCq.range);
+  }
+
+  public static StoredTabletFile of(final Text metadataEntry) {
+    return new StoredTabletFile(Objects.requireNonNull(metadataEntry).toString());
+  }
+
+  public static StoredTabletFile of(final String metadataEntry) {
+    return new StoredTabletFile(metadataEntry);
+  }
+
+  public static StoredTabletFile of(final URI path, Range range) {
+    return of(new Path(Objects.requireNonNull(path)), range);
+  }
+
+  public static StoredTabletFile of(final Path path, Range range) {
+    return new StoredTabletFile(new TabletFileCq(Objects.requireNonNull(path), range));
+  }
+
+  public static StoredTabletFile of(final URI path) {
+    return of(path, new Range());
+  }
+
+  public static StoredTabletFile of(final Path path) {
+    return of(path, new Range());
+  }
+
+  private static final Gson gson = ByteArrayToBase64TypeAdapter.createBase64Gson();
+
+  private static TabletFileCq deserialize(String json) {
+    final TabletFileCqMetadataGson metadata =
+        gson.fromJson(Objects.requireNonNull(json), TabletFileCqMetadataGson.class);
+
+    // Check each field and provide better error messages if null as all fields should be set
+    Objects.requireNonNull(metadata.path, "Serialized StoredTabletFile path must not be null");
+    Objects.requireNonNull(metadata.startRow,
+        "Serialized StoredTabletFile range startRow must not be null");
+    Objects.requireNonNull(metadata.endRow,
+        "Serialized StoredTabletFile range endRow must not be null");
+
+    // Recreate the exact Range that was originally stored in Metadata. Stored ranges are originally
+    // constructed with inclusive/exclusive for the start and end key inclusivity settings.
+    // (Except for Ranges with no start/endkey as then the inclusivity flags do not matter)
+    // The ranges must match the format of KeyExtent.toDataRange()
+    //
+    // With this particular constructor, when setting the startRowInclusive to true and
+    // endRowInclusive to false, both the start and end row values will be taken as is
+    // and not modified and will recreate the original Range.
+    //
+    // This constructor will always set the resulting inclusivity of the Range to be true for the
+    // start row and false for end row regardless of what the startRowInclusive and endRowInclusive
+    // flags are set to.
+    return new TabletFileCq(new Path(URI.create(metadata.path)),
+        new Range(decodeRow(metadata.startRow), true, decodeRow(metadata.endRow), false));
+  }
+
+  public static String serialize(String path) {
+    return serialize(path, new Range());
+  }
+
+  public static String serialize(String path, Range range) {
+    requireKeyExtentDataRange(range);
+    final TabletFileCqMetadataGson metadata = new TabletFileCqMetadataGson();
+    metadata.path = Objects.requireNonNull(path);
+    metadata.startRow = encodeRow(range.getStartKey());
+    metadata.endRow = encodeRow(range.getEndKey());
+
+    return gson.toJson(metadata);
+  }
+
+  private static String serialize(TabletFileCq tabletFileCq) {
+    return serialize(Objects.requireNonNull(tabletFileCq).path.toString(), tabletFileCq.range);
+  }
+
+  /**
+   * Helper methods to encode and decode rows in a range to/from byte arrays. Null rows will just be
+   * returned as an empty byte array
+   **/
+
+  private static byte[] encodeRow(final Key key) {
+    final Text row = key != null ? key.getRow() : null;
+    if (row != null) {
+      try (ByteArrayOutputStream baos = new ByteArrayOutputStream();
+          DataOutputStream dos = new DataOutputStream(baos)) {
+        row.write(dos);
+        dos.close();
+        return baos.toByteArray();
+      } catch (IOException e) {
+        throw new UncheckedIOException(e);
+      }
+    }
+    // Empty byte array means null row
+    return new byte[0];
+  }
+
+  private static Text decodeRow(byte[] serialized) {
+    // Empty byte array means null row
+    if (serialized.length == 0) {
+      return null;
+    }
+
+    try (DataInputBuffer buffer = new DataInputBuffer()) {
+      final Text row = new Text();
+      buffer.reset(serialized, serialized.length);
+      row.readFields(buffer);
+      return row;
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  /**
+   * Quick validation to see if value has been converted by checking if the candidate looks like
+   * json by checking the candidate starts with "{" and ends with "}".
+   *
+   * @param candidate a possible file: reference.
+   * @return false if a likely a json object, true if not a likely json object
+   */
+  @VisibleForTesting
+  public static boolean fileNeedsConversion(@NonNull final String candidate) {
+    String trimmed = candidate.trim();
+    return !trimmed.startsWith("{") || !trimmed.endsWith("}");
+  }
+
+  private static class TabletFileCq {
+    public final Path path;
+    public final Range range;
+
+    public TabletFileCq(Path path, Range range) {
+      this.path = Objects.requireNonNull(path);
+      this.range = Objects.requireNonNull(range);
+    }
+  }
+
+  private static class TabletFileCqMetadataGson {
+    private String path;
+    private byte[] startRow;
+    private byte[] endRow;
+  }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/SuspendingTServer.java b/core/src/main/java/org/apache/accumulo/core/metadata/SuspendingTServer.java
index 8b307ca..e481369 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/SuspendingTServer.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/SuspendingTServer.java
@@ -19,30 +19,34 @@
 package org.apache.accumulo.core.metadata;
 
 import java.util.Objects;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.util.HostAndPort;
+import org.apache.accumulo.core.util.time.SteadyTime;
+
+import com.google.common.net.HostAndPort;
 
 /**
  * For a suspended tablet, the time of suspension and the server it was suspended from.
  */
 public class SuspendingTServer {
   public final HostAndPort server;
-  public final long suspensionTime;
+  public final SteadyTime suspensionTime;
 
-  SuspendingTServer(HostAndPort server, long suspensionTime) {
+  SuspendingTServer(HostAndPort server, SteadyTime suspensionTime) {
     this.server = Objects.requireNonNull(server);
-    this.suspensionTime = suspensionTime;
+    this.suspensionTime = Objects.requireNonNull(suspensionTime);
   }
 
   public static SuspendingTServer fromValue(Value value) {
     String valStr = value.toString();
     String[] parts = valStr.split("[|]", 2);
-    return new SuspendingTServer(HostAndPort.fromString(parts[0]), Long.parseLong(parts[1]));
+    return new SuspendingTServer(HostAndPort.fromString(parts[0]),
+        SteadyTime.from(Long.parseLong(parts[1]), TimeUnit.MILLISECONDS));
   }
 
-  public static Value toValue(TServerInstance tServer, long suspensionTime) {
-    return new Value(tServer.getHostPort() + "|" + suspensionTime);
+  public static Value toValue(TServerInstance tServer, SteadyTime suspensionTime) {
+    return new Value(tServer.getHostPort() + "|" + suspensionTime.getMillis());
   }
 
   @Override
@@ -51,7 +55,7 @@
       return false;
     }
     SuspendingTServer rhs = (SuspendingTServer) rhsObject;
-    return server.equals(rhs.server) && suspensionTime == rhs.suspensionTime;
+    return server.equals(rhs.server) && suspensionTime.equals(rhs.suspensionTime);
   }
 
   @Override
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/TServerInstance.java b/core/src/main/java/org/apache/accumulo/core/metadata/TServerInstance.java
index 2bf8ca4..9e26102 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/TServerInstance.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/TServerInstance.java
@@ -22,9 +22,10 @@
 
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.util.AddressUtil;
-import org.apache.accumulo.core.util.HostAndPort;
 import org.apache.hadoop.io.Text;
 
+import com.google.common.net.HostAndPort;
+
 /**
  * A tablet is assigned to a tablet server at the given address as long as it is alive and well.
  * When the tablet server is restarted, the instance information it advertises will change.
@@ -61,11 +62,11 @@
   }
 
   public TServerInstance(String address, long session) {
-    this(AddressUtil.parseAddress(address, false), Long.toHexString(session));
+    this(AddressUtil.parseAddress(address), Long.toHexString(session));
   }
 
   public TServerInstance(Value address, Text session) {
-    this(AddressUtil.parseAddress(new String(address.get(), UTF_8), false), session.toString());
+    this(AddressUtil.parseAddress(new String(address.get(), UTF_8)), session.toString());
   }
 
   @Override
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/TabletFile.java b/core/src/main/java/org/apache/accumulo/core/metadata/TabletFile.java
index f216e24..8b90bd7 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/TabletFile.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/TabletFile.java
@@ -18,224 +18,43 @@
  */
 package org.apache.accumulo.core.metadata;
 
-import static org.apache.accumulo.core.Constants.HDFS_TABLES_DIR;
-
-import java.net.URI;
-import java.util.Objects;
-
-import org.apache.accumulo.core.data.TableId;
-import org.apache.commons.lang3.StringUtils;
+import org.apache.accumulo.core.data.Range;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.base.Preconditions;
 
 /**
- * Object representing a tablet file that may exist in the metadata table. This class is used for
- * reading and opening tablet files. It is also used when inserting new tablet files. When a new
- * file is inserted, the {@link #insert()} method is called and returns a {@link StoredTabletFile}
- * For situations where a tablet file needs to be updated or deleted in the metadata, a
- * {@link StoredTabletFile} is required.
- * <p>
- * As of 2.1, Tablet file paths should now be only absolute URIs with the removal of relative paths
- * in Upgrader9to10.upgradeRelativePaths()
+ * An interface that represents different types of file references that are handled by code that
+ * processes tablet files.
  */
-public class TabletFile implements Comparable<TabletFile> {
-
-  public static class FileParts {
-
-    // parts of an absolute URI, like "hdfs://1.2.3.4/accumulo/tables/2a/t-0003/C0004.rf"
-    // volume: hdfs://1.2.3.4/accumulo
-    // tableId: 2a
-    // tabletDir: t-0003
-    // fileName: C0004.rf
-    // normalizedPath: hdfs://1.2.3.4/accumulo/tables/2a/t-0003/C0004.rf
-    private final String volume;
-    private final TableId tableId;
-    private final String tabletDir;
-    private final String fileName;
-    private final String normalizedPath;
-
-    public FileParts(String volume, TableId tableId, String tabletDir, String fileName,
-        String normalizedPath) {
-      this.volume = volume;
-      this.tableId = tableId;
-      this.tabletDir = tabletDir;
-      this.fileName = fileName;
-      this.normalizedPath = normalizedPath;
-    }
-
-    public String getVolume() {
-      return volume;
-    }
-
-    public TableId getTableId() {
-      return tableId;
-    }
-
-    public String getTabletDir() {
-      return tabletDir;
-    }
-
-    public String getFileName() {
-      return fileName;
-    }
-
-    public String getNormalizedPath() {
-      return normalizedPath;
-    }
-
-  }
-
-  private static String constructErrorMsg(Path filePath) {
-    return "Missing or invalid part of tablet file metadata entry: " + filePath;
-  }
-
-  public static FileParts parsePath(Path filePath) {
-    // File name construct: <volume>/<tablePath>/<tableId>/<tablet>/<file>
-    // Example: hdfs://namenode:9020/accumulo/tables/1/default_tablet/F00001.rf
-    final URI uri = filePath.toUri();
-
-    // validate that this is a fully qualified uri
-    Preconditions.checkArgument(uri.getScheme() != null, constructErrorMsg(filePath));
-
-    final String path = uri.getPath(); // ex: /accumulo/tables/1/default_tablet/F00001.rf
-    final String[] parts = path.split("/");
-    final int numParts = parts.length; // should contain tables, 1, default_tablet, F00001.rf
-
-    if (numParts < 4) {
-      throw new IllegalArgumentException(constructErrorMsg(filePath));
-    }
-
-    final String fileName = parts[numParts - 1];
-    final String tabletDirectory = parts[numParts - 2];
-    final TableId tableId = TableId.of(parts[numParts - 3]);
-    final String tablesPath = parts[numParts - 4];
-
-    // determine where file path starts, the rest is the volume
-    final String computedFilePath =
-        HDFS_TABLES_DIR + "/" + tableId.canonical() + "/" + tabletDirectory + "/" + fileName;
-    final String uriString = uri.toString();
-    int idx = uriString.lastIndexOf(computedFilePath);
-
-    if (idx == -1) {
-      throw new IllegalArgumentException(constructErrorMsg(filePath));
-    }
-
-    // The volume is the beginning portion of the uri up to the start
-    // of the file path.
-    final String volume = uriString.substring(0, idx);
-
-    if (StringUtils.isBlank(fileName) || StringUtils.isBlank(tabletDirectory)
-        || StringUtils.isBlank(tablesPath) || StringUtils.isBlank(volume)) {
-      throw new IllegalArgumentException(constructErrorMsg(filePath));
-    }
-    ValidationUtil.validateFileName(fileName);
-    Preconditions.checkArgument(tablesPath.equals(HDFS_TABLES_DIR_NAME),
-        "tables directory name is not " + HDFS_TABLES_DIR_NAME + ", is " + tablesPath);
-
-    final String normalizedPath = volume + computedFilePath;
-
-    if (!normalizedPath.equals(uriString)) {
-      throw new RuntimeException("Error parsing file path, " + normalizedPath + " != " + uriString);
-    }
-
-    return new FileParts(volume, tableId, tabletDirectory, fileName, normalizedPath);
-
-  }
-
-  protected final Path metaPath;
-  private final FileParts parts;
-
-  private static final Logger log = LoggerFactory.getLogger(TabletFile.class);
-  private static final String HDFS_TABLES_DIR_NAME = HDFS_TABLES_DIR.substring(1);
+public interface TabletFile {
 
   /**
-   * Construct new tablet file using a Path. Used in the case where we had to use Path object to
-   * qualify an absolute path or create a new file.
+   * Returns the fileName of the TabletFile. The value return is the name itself and not the entire
+   * path. For example, if the full path for a TabletFile is
+   * 'hdfs://nn1/accumulo/tables/5a/t-0001/F0002.rf', this method returns 'F0002.rf'.
    */
-  public TabletFile(Path metaPath) {
-    this.metaPath = Objects.requireNonNull(metaPath);
-    log.trace("Parsing TabletFile from {}", metaPath);
-    parts = parsePath(metaPath);
-  }
-
-  public String getVolume() {
-    return parts.getVolume();
-  }
-
-  public TableId getTableId() {
-    return parts.getTableId();
-  }
-
-  public String getTabletDir() {
-    return parts.getTabletDir();
-  }
-
-  public String getFileName() {
-    return parts.getFileName();
-  }
+  String getFileName();
 
   /**
-   * Return a string for opening and reading the tablet file. Doesn't have to be exact string in
-   * metadata.
+   * Returns the full path for the TabletFile on the file system. The path may be normalized
+   * depending on the specific implementation. For example, a path in hdfs would be returned as
+   * 'hdfs://nn1/accumulo/tables/5a/t-0001/F0002.rf'
    */
-  public String getPathStr() {
-    return parts.getNormalizedPath();
-  }
+  Path getPath();
 
   /**
-   * Return a string for inserting a new tablet file.
+   * @return The range of the TabletFile
+   *
    */
-  public String getMetaInsert() {
-    return parts.getNormalizedPath();
-  }
+  Range getRange();
 
   /**
-   * Return a new Text object of {@link #getMetaInsert()}
+   * @return True if this file is fenced by a range
+   *
    */
-  public Text getMetaInsertText() {
-    return new Text(getMetaInsert());
-  }
+  boolean hasRange();
 
   /**
-   * New file was written to metadata so return a StoredTabletFile
+   * @return a string with the filename and row range if there is one.
    */
-  public StoredTabletFile insert() {
-    return new StoredTabletFile(parts.getNormalizedPath());
-  }
-
-  public Path getPath() {
-    return metaPath;
-  }
-
-  @Override
-  public int compareTo(TabletFile o) {
-    if (equals(o)) {
-      return 0;
-    } else {
-      return parts.getNormalizedPath().compareTo(o.parts.getNormalizedPath());
-    }
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (obj instanceof TabletFile) {
-      TabletFile that = (TabletFile) obj;
-      return parts.getNormalizedPath().equals(that.parts.getNormalizedPath());
-    }
-    return false;
-  }
-
-  @Override
-  public int hashCode() {
-    return parts.getNormalizedPath().hashCode();
-  }
-
-  @Override
-  public String toString() {
-    return parts.getNormalizedPath();
-  }
+  String toMinimalString();
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/TabletLocationState.java b/core/src/main/java/org/apache/accumulo/core/metadata/TabletLocationState.java
index 0a70594..4c2421b 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/TabletLocationState.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/TabletLocationState.java
@@ -28,6 +28,7 @@
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
+import org.apache.accumulo.core.tabletserver.log.LogEntry;
 import org.apache.accumulo.core.util.TextUtil;
 import org.apache.hadoop.io.Text;
 
@@ -57,8 +58,7 @@
   }
 
   public TabletLocationState(KeyExtent extent, Location future, Location current, Location last,
-      SuspendingTServer suspend, Collection<Collection<String>> walogs, boolean chopped)
-      throws BadLocationStateException {
+      SuspendingTServer suspend, Collection<LogEntry> walogs) throws BadLocationStateException {
     this.extent = Objects.requireNonNull(extent);
     this.future = validateLocation(future, TabletMetadata.LocationType.FUTURE);
     this.current = validateLocation(current, TabletMetadata.LocationType.CURRENT);
@@ -68,7 +68,6 @@
       walogs = Collections.emptyList();
     }
     this.walogs = walogs;
-    this.chopped = chopped;
     if (hasCurrent() && hasFuture()) {
       throw new BadLocationStateException(
           extent + " is both assigned and hosted, which should never happen: " + this,
@@ -81,8 +80,7 @@
   public final Location current;
   public final Location last;
   public final SuspendingTServer suspend;
-  public final Collection<Collection<String>> walogs;
-  public final boolean chopped;
+  public final Collection<LogEntry> walogs;
 
   public TServerInstance getCurrentServer() {
     return serverInstance(current);
@@ -151,7 +149,7 @@
 
   @Override
   public String toString() {
-    return extent + "@(" + future + "," + current + "," + last + ")" + (chopped ? " chopped" : "");
+    return extent + "@(" + future + "," + current + "," + last + ")";
   }
 
   private static Location validateLocation(final Location location,
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/UnreferencedTabletFile.java b/core/src/main/java/org/apache/accumulo/core/metadata/UnreferencedTabletFile.java
new file mode 100644
index 0000000..3d5c1d4
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/UnreferencedTabletFile.java
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.metadata;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Objects;
+
+import org.apache.accumulo.core.data.Range;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+/**
+ * A file that is not intended to be added to a tablet as a reference, within the scope of the code
+ * using this class, but needs to be passed to code that processes tablet files. These files could
+ * be temp files or files directly created by a user for bulk import. The file may ultimately be
+ * added to a tablet later as a new file reference, but within a different scope (process, thread,
+ * code block, method, etc.) that uses a different class to represent the file in that scope.
+ *
+ * Unlike {@link ReferencedTabletFile}, this class does not perform any validation or normalization
+ * on the provided path.
+ *
+ * @since 3.0.0
+ */
+public class UnreferencedTabletFile extends AbstractTabletFile<UnreferencedTabletFile> {
+
+  private final String fileName; // C0004.rf
+
+  public UnreferencedTabletFile(FileSystem fs, Path path) {
+    this(fs, path, new Range());
+  }
+
+  public UnreferencedTabletFile(FileSystem fs, Path path, Range range) {
+    super(Objects.requireNonNull(fs).makeQualified(Objects.requireNonNull(path)), range);
+    this.fileName = path.getName();
+    ValidationUtil.validateFileName(fileName);
+  }
+
+  @Override
+  public String getFileName() {
+    return fileName;
+  }
+
+  @Override
+  public int compareTo(UnreferencedTabletFile o) {
+    if (equals(o)) {
+      return 0;
+    } else {
+      return path.compareTo(o.path);
+    }
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj instanceof UnreferencedTabletFile) {
+      UnreferencedTabletFile that = (UnreferencedTabletFile) obj;
+      return path.equals(that.path);
+    }
+    return false;
+  }
+
+  @Override
+  public int hashCode() {
+    return path.hashCode();
+  }
+
+  @Override
+  public String toString() {
+    return path.toString();
+  }
+
+  public static UnreferencedTabletFile of(FileSystem fs, File file) {
+    return new UnreferencedTabletFile(fs, new Path(Objects.requireNonNull(file).toString()));
+  }
+
+  public static UnreferencedTabletFile ofRanged(FileSystem fs, File file, Range range) {
+    return new UnreferencedTabletFile(fs, new Path(Objects.requireNonNull(file).toString()), range);
+  }
+
+  public static UnreferencedTabletFile of(FileSystem fs, Path path) {
+    return new UnreferencedTabletFile(fs, path);
+  }
+
+  public static UnreferencedTabletFile ofRanged(FileSystem fs, Path path, Range range) {
+    return new UnreferencedTabletFile(fs, path, range);
+  }
+
+  public static UnreferencedTabletFile of(Configuration conf, Path path) throws IOException {
+    return new UnreferencedTabletFile(Objects.requireNonNull(path).getFileSystem(conf), path);
+  }
+
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/ValidationUtil.java b/core/src/main/java/org/apache/accumulo/core/metadata/ValidationUtil.java
index db60a89..26d49e5 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/ValidationUtil.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/ValidationUtil.java
@@ -43,7 +43,7 @@
   }
 
   public static ReferenceFile validate(ReferenceFile reference) {
-    validate(new Path(reference.getMetadataEntry()));
+    validate(new Path(reference.getMetadataPath()));
     return reference;
   }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/schema/Ample.java b/core/src/main/java/org/apache/accumulo/core/metadata/schema/Ample.java
index f7232d7..b212356 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/schema/Ample.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/schema/Ample.java
@@ -20,24 +20,23 @@
 
 import java.util.Collection;
 import java.util.Iterator;
-import java.util.UUID;
 import java.util.stream.Stream;
 
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock;
 import org.apache.accumulo.core.gc.GcCandidate;
 import org.apache.accumulo.core.gc.ReferenceFile;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.RootTable;
-import org.apache.accumulo.core.metadata.ScanServerRefTabletFile;
+import org.apache.accumulo.core.lock.ServiceLock;
+import org.apache.accumulo.core.metadata.AccumuloTable;
+import org.apache.accumulo.core.metadata.ReferencedTabletFile;
+import org.apache.accumulo.core.metadata.ScanServerRefStore;
 import org.apache.accumulo.core.metadata.StoredTabletFile;
 import org.apache.accumulo.core.metadata.TServerInstance;
-import org.apache.accumulo.core.metadata.TabletFile;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
 import org.apache.accumulo.core.tabletserver.log.LogEntry;
+import org.apache.accumulo.core.util.time.SteadyTime;
 import org.apache.hadoop.io.Text;
 
 /**
@@ -74,8 +73,8 @@
    */
   public enum DataLevel {
     ROOT(null, null),
-    METADATA(RootTable.NAME, RootTable.ID),
-    USER(MetadataTable.NAME, MetadataTable.ID);
+    METADATA(AccumuloTable.ROOT.tableName(), AccumuloTable.ROOT.tableId()),
+    USER(AccumuloTable.METADATA.tableName(), AccumuloTable.METADATA.tableId());
 
     private final String table;
     private final TableId id;
@@ -106,9 +105,9 @@
     }
 
     public static DataLevel of(TableId tableId) {
-      if (tableId.equals(RootTable.ID)) {
+      if (tableId.equals(AccumuloTable.ROOT.tableId())) {
         return DataLevel.ROOT;
-      } else if (tableId.equals(MetadataTable.ID)) {
+      } else if (tableId.equals(AccumuloTable.METADATA.tableId())) {
         return DataLevel.METADATA;
       } else {
         return DataLevel.USER;
@@ -241,7 +240,7 @@
   /**
    * Return an encoded delete marker Mutation to delete the specified TabletFile path. A
    * ReferenceFile is used for the parameter because the Garbage Collector is optimized to store a
-   * directory for Tablet File. Otherwise, a {@link TabletFile} object could be used. The
+   * directory for Tablet File. Otherwise, a {@link ReferencedTabletFile} object could be used. The
    * tabletFilePathToRemove is validated and normalized before creating the mutation.
    *
    * @param tabletFilePathToRemove String full path of the TabletFile
@@ -268,13 +267,13 @@
   interface TabletMutator {
     TabletMutator putPrevEndRow(Text per);
 
-    TabletMutator putFile(TabletFile path, DataFileValue dfv);
+    TabletMutator putFile(ReferencedTabletFile path, DataFileValue dfv);
 
     TabletMutator putFile(StoredTabletFile path, DataFileValue dfv);
 
     TabletMutator deleteFile(StoredTabletFile path);
 
-    TabletMutator putScan(TabletFile path);
+    TabletMutator putScan(StoredTabletFile path);
 
     TabletMutator deleteScan(StoredTabletFile path);
 
@@ -292,19 +291,15 @@
 
     TabletMutator putWal(LogEntry logEntry);
 
-    TabletMutator deleteWal(String wal);
-
-    TabletMutator deleteWal(LogEntry logEntry);
+    TabletMutator deleteWal(LogEntry wal);
 
     TabletMutator putTime(MetadataTime time);
 
-    TabletMutator putBulkFile(TabletFile bulkref, long tid);
+    TabletMutator putBulkFile(ReferencedTabletFile bulkref, long tid);
 
-    TabletMutator deleteBulkFile(TabletFile bulkref);
+    TabletMutator deleteBulkFile(StoredTabletFile bulkref);
 
-    TabletMutator putChopped();
-
-    TabletMutator putSuspension(TServerInstance tserver, long suspensionTime);
+    TabletMutator putSuspension(TServerInstance tserver, SteadyTime suspensionTime);
 
     TabletMutator deleteSuspension();
 
@@ -329,43 +324,6 @@
   }
 
   /**
-   * Insert ScanServer references to Tablet files
-   *
-   * @param scanRefs set of scan server ref table file objects
-   */
-  default void putScanServerFileReferences(Collection<ScanServerRefTabletFile> scanRefs) {
-    throw new UnsupportedOperationException();
-  }
-
-  /**
-   * Get ScanServer references to Tablet files
-   *
-   * @return stream of scan server references
-   */
-  default Stream<ScanServerRefTabletFile> getScanServerFileReferences() {
-    throw new UnsupportedOperationException();
-  }
-
-  /**
-   * Delete the set of scan server references
-   *
-   * @param refsToDelete set of scan server references to delete
-   */
-  default void deleteScanServerFileReferences(Collection<ScanServerRefTabletFile> refsToDelete) {
-    throw new UnsupportedOperationException();
-  }
-
-  /**
-   * Delete scan server references for this server
-   *
-   * @param serverAddress address of server, cannot be null
-   * @param serverSessionId server session id, cannot be null
-   */
-  default void deleteScanServerFileReferences(String serverAddress, UUID serverSessionId) {
-    throw new UnsupportedOperationException();
-  }
-
-  /**
    * Create a Bulk Load In Progress flag in the metadata table
    *
    * @param path The bulk directory filepath
@@ -384,6 +342,10 @@
     throw new UnsupportedOperationException();
   }
 
+  default ScanServerRefStore scanServerRefs() {
+    throw new UnsupportedOperationException();
+  }
+
   /**
    * Remove all the Bulk Load transaction ids from a given table's metadata
    *
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/schema/DataFileValue.java b/core/src/main/java/org/apache/accumulo/core/metadata/schema/DataFileValue.java
index 67c5e15..fe770d9 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/schema/DataFileValue.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/schema/DataFileValue.java
@@ -21,6 +21,8 @@
 import static java.nio.charset.StandardCharsets.UTF_8;
 
 import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.iteratorsImpl.system.InterruptibleIterator;
+import org.apache.accumulo.core.iteratorsImpl.system.TimeSettingIterator;
 
 public class DataFileValue {
   private long size;
@@ -114,4 +116,23 @@
     }
     this.time = time;
   }
+
+  /**
+   * @return true if {@link #wrapFileIterator} would wrap a given iterator, false otherwise.
+   */
+  public boolean willWrapIterator() {
+    return isTimeSet();
+  }
+
+  /**
+   * Use per file information from the metadata table to wrap the raw iterator over a file with
+   * iterators that may take action based on data set in the metadata table.
+   */
+  public InterruptibleIterator wrapFileIterator(InterruptibleIterator iter) {
+    if (isTimeSet()) {
+      return new TimeSettingIterator(iter, getTime());
+    } else {
+      return iter;
+    }
+  }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/schema/ExternalCompactionFinalState.java b/core/src/main/java/org/apache/accumulo/core/metadata/schema/ExternalCompactionFinalState.java
index 9212ccd..421a707 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/schema/ExternalCompactionFinalState.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/schema/ExternalCompactionFinalState.java
@@ -18,6 +18,8 @@
  */
 package org.apache.accumulo.core.metadata.schema;
 
+import static org.apache.accumulo.core.util.LazySingletons.GSON;
+
 import java.util.Base64;
 
 import org.apache.accumulo.core.data.TableId;
@@ -26,12 +28,9 @@
 import org.apache.hadoop.io.Text;
 
 import com.google.common.base.Preconditions;
-import com.google.gson.Gson;
 
 public class ExternalCompactionFinalState {
 
-  private static final Gson GSON = new Gson();
-
   public enum FinalState {
     FINISHED, FAILED
   }
@@ -123,11 +122,11 @@
     jd.fileSize = fileSize;
     jd.entries = fileEntries;
     jd.extent = new Extent(extent);
-    return GSON.toJson(jd);
+    return GSON.get().toJson(jd);
   }
 
   public static ExternalCompactionFinalState fromJson(ExternalCompactionId ecid, String json) {
-    JsonData jd = GSON.fromJson(json, JsonData.class);
+    JsonData jd = GSON.get().fromJson(json, JsonData.class);
     return new ExternalCompactionFinalState(ecid, jd.extent.toKeyExtent(),
         FinalState.valueOf(jd.state), jd.fileSize, jd.entries);
   }
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/schema/ExternalCompactionMetadata.java b/core/src/main/java/org/apache/accumulo/core/metadata/schema/ExternalCompactionMetadata.java
index 71837f0..4c7e429 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/schema/ExternalCompactionMetadata.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/schema/ExternalCompactionMetadata.java
@@ -20,27 +20,24 @@
 
 import static java.util.stream.Collectors.toList;
 import static java.util.stream.Collectors.toSet;
+import static org.apache.accumulo.core.util.LazySingletons.GSON;
 
 import java.util.List;
 import java.util.Objects;
 import java.util.Set;
 
+import org.apache.accumulo.core.metadata.ReferencedTabletFile;
 import org.apache.accumulo.core.metadata.StoredTabletFile;
-import org.apache.accumulo.core.metadata.TabletFile;
 import org.apache.accumulo.core.spi.compaction.CompactionExecutorId;
 import org.apache.accumulo.core.spi.compaction.CompactionKind;
 import org.apache.accumulo.core.util.compaction.CompactionExecutorIdImpl;
-import org.apache.hadoop.fs.Path;
-
-import com.google.gson.Gson;
+import org.apache.accumulo.core.util.compaction.DeprecatedCompactionKind;
 
 public class ExternalCompactionMetadata {
 
-  private static final Gson GSON = new Gson();
-
   private final Set<StoredTabletFile> jobFiles;
   private final Set<StoredTabletFile> nextFiles;
-  private final TabletFile compactTmpName;
+  private final ReferencedTabletFile compactTmpName;
   private final String compactorId;
   private final CompactionKind kind;
   private final short priority;
@@ -50,11 +47,11 @@
   private final Long compactionId;
 
   public ExternalCompactionMetadata(Set<StoredTabletFile> jobFiles, Set<StoredTabletFile> nextFiles,
-      TabletFile compactTmpName, String compactorId, CompactionKind kind, short priority,
+      ReferencedTabletFile compactTmpName, String compactorId, CompactionKind kind, short priority,
       CompactionExecutorId ceid, boolean propagateDeletes, boolean initiallySelectedAll,
       Long compactionId) {
     if (!initiallySelectedAll && !propagateDeletes
-        && (kind == CompactionKind.SELECTOR || kind == CompactionKind.USER)) {
+        && (kind == DeprecatedCompactionKind.SELECTOR || kind == CompactionKind.USER)) {
       throw new IllegalArgumentException(
           "When user or selector compactions do not propagate deletes, it's expected that all "
               + "files were selected initially.");
@@ -79,7 +76,7 @@
     return nextFiles;
   }
 
-  public TabletFile getCompactTmpName() {
+  public ReferencedTabletFile getCompactTmpName() {
     return compactTmpName;
   }
 
@@ -129,10 +126,9 @@
   public String toJson() {
     GSonData jData = new GSonData();
 
-    jData.inputs = jobFiles.stream().map(StoredTabletFile::getMetaUpdateDelete).collect(toList());
-    jData.nextFiles =
-        nextFiles.stream().map(StoredTabletFile::getMetaUpdateDelete).collect(toList());
-    jData.tmp = compactTmpName.getMetaInsert();
+    jData.inputs = jobFiles.stream().map(StoredTabletFile::getMetadata).collect(toList());
+    jData.nextFiles = nextFiles.stream().map(StoredTabletFile::getMetadata).collect(toList());
+    jData.tmp = compactTmpName.insert().getMetadata();
     jData.compactor = compactorId;
     jData.kind = kind.name();
     jData.executorId = ((CompactionExecutorIdImpl) ceid).getExternalName();
@@ -140,18 +136,19 @@
     jData.propDels = propagateDeletes;
     jData.selectedAll = initiallySelectedAll;
     jData.compactionId = compactionId;
-    return GSON.toJson(jData);
+    return GSON.get().toJson(jData);
   }
 
   public static ExternalCompactionMetadata fromJson(String json) {
-    GSonData jData = GSON.fromJson(json, GSonData.class);
+    GSonData jData = GSON.get().fromJson(json, GSonData.class);
 
     return new ExternalCompactionMetadata(
         jData.inputs.stream().map(StoredTabletFile::new).collect(toSet()),
         jData.nextFiles.stream().map(StoredTabletFile::new).collect(toSet()),
-        new TabletFile(new Path(jData.tmp)), jData.compactor, CompactionKind.valueOf(jData.kind),
-        jData.priority, CompactionExecutorIdImpl.externalId(jData.executorId), jData.propDels,
-        jData.selectedAll, jData.compactionId);
+        StoredTabletFile.of(jData.tmp).getTabletFile(), jData.compactor,
+        CompactionKind.valueOf(jData.kind), jData.priority,
+        CompactionExecutorIdImpl.externalId(jData.executorId), jData.propDels, jData.selectedAll,
+        jData.compactionId);
   }
 
   @Override
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/schema/LinkingIterator.java b/core/src/main/java/org/apache/accumulo/core/metadata/schema/LinkingIterator.java
index 5da06b8..7c8bf7b 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/schema/LinkingIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/schema/LinkingIterator.java
@@ -18,8 +18,8 @@
  */
 package org.apache.accumulo.core.metadata.schema;
 
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
 
 import java.util.Iterator;
 import java.util.function.Function;
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataSchema.java b/core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataSchema.java
index 5bb4e3a..e4356c5 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataSchema.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataSchema.java
@@ -18,14 +18,11 @@
  */
 package org.apache.accumulo.core.metadata.schema;
 
-import static com.google.common.base.Preconditions.checkArgument;
 import static java.nio.charset.StandardCharsets.UTF_8;
-import static java.util.Objects.requireNonNull;
 
 import java.util.regex.Pattern;
 
 import org.apache.accumulo.core.client.admin.TimeType;
-import org.apache.accumulo.core.data.ArrayByteSequence;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.PartialKey;
@@ -329,15 +326,27 @@
      * data for the current tablet, so that they are safe to merge
      */
     public static class ChoppedColumnFamily {
-      public static final String STR_NAME = "chopped";
-      public static final Text NAME = new Text(STR_NAME);
-      public static final ColumnFQ CHOPPED_COLUMN = new ColumnFQ(NAME, new Text(STR_NAME));
+      // kept to support upgrades to 3.1; name is used for both col fam and col qual
+      @Deprecated(since = "3.1.0")
+      public static final Text NAME = new Text("chopped");
     }
 
     public static class ExternalCompactionColumnFamily {
       public static final String STR_NAME = "ecomp";
       public static final Text NAME = new Text(STR_NAME);
     }
+
+    /**
+     * Column family for indicating that the files in a tablet contain fenced files that have been
+     * merged from other tablets during a merge operation. This is used to support resuming a failed
+     * merge operation.
+     */
+    public static class MergedColumnFamily {
+      public static final String STR_NAME = "merged";
+      public static final Text NAME = new Text(STR_NAME);
+      public static final ColumnFQ MERGED_COLUMN = new ColumnFQ(NAME, new Text(STR_NAME));
+      public static final Value MERGED_VALUE = new Value("merged");
+    }
   }
 
   /**
@@ -422,58 +431,6 @@
 
   }
 
-  /**
-   * Holds references to files that need replication
-   *
-   * <pre>
-   * <code>
-   * ~replhdfs://localhost:8020/accumulo/wal/tserver+port/WAL stat:local_table_id [] -&gt; protobuf
-   * </code>
-   * </pre>
-   */
-  public static class ReplicationSection {
-    public static final Text COLF = new Text("stat");
-    private static final ArrayByteSequence COLF_BYTE_SEQ = new ArrayByteSequence(COLF.toString());
-    private static final Section section =
-        new Section(RESERVED_PREFIX + "repl", true, RESERVED_PREFIX + "repm", false);
-
-    public static Range getRange() {
-      return section.getRange();
-    }
-
-    public static String getRowPrefix() {
-      return section.getRowPrefix();
-    }
-
-    /**
-     * Extract the table ID from the colfam
-     *
-     * @param k Key to extract from
-     */
-    public static TableId getTableId(Key k) {
-      requireNonNull(k);
-      return TableId.of(k.getColumnQualifier().toString());
-    }
-
-    /**
-     * Extract the file name from the row suffix into the given {@link Text}
-     *
-     * @param k Key to extract from
-     * @param buff Text to place file name into
-     */
-    public static void getFile(Key k, Text buff) {
-      requireNonNull(k);
-      requireNonNull(buff);
-      checkArgument(COLF_BYTE_SEQ.equals(k.getColumnFamilyData()),
-          "Given metadata replication status key with incorrect colfam");
-
-      k.getRow(buff);
-
-      buff.set(buff.getBytes(), section.getRowPrefix().length(),
-          buff.getLength() - section.getRowPrefix().length());
-    }
-  }
-
   public static class ExternalCompactionSection {
     private static final Section section =
         new Section(RESERVED_PREFIX + "ecomp", true, RESERVED_PREFIX + "ecomq", false);
@@ -486,31 +443,4 @@
       return section.getRowPrefix();
     }
   }
-
-  public static class ScanServerFileReferenceSection {
-    private static final Section section =
-        new Section(RESERVED_PREFIX + "scanfileref", true, RESERVED_PREFIX + "scanfilereg", false);
-
-    public static Range getRange() {
-      return section.getRange();
-    }
-
-    public static String getRowPrefix() {
-      return section.getRowPrefix();
-    }
-  }
-
-  @Deprecated(since = "2.1")
-  public static class OldScanServerFileReferenceSection {
-    private static final Section section =
-        new Section(RESERVED_PREFIX + "sserv", true, RESERVED_PREFIX + "sserx", false);
-
-    public static Range getRange() {
-      return section.getRange();
-    }
-
-    public static String getRowPrefix() {
-      return section.getRowPrefix();
-    }
-  }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/schema/RootTabletMetadata.java b/core/src/main/java/org/apache/accumulo/core/metadata/schema/RootTabletMetadata.java
index 3f01eed..cc16bb6 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/schema/RootTabletMetadata.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/schema/RootTabletMetadata.java
@@ -20,6 +20,7 @@
 
 import static com.google.common.base.Preconditions.checkArgument;
 import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.core.util.LazySingletons.GSON;
 
 import java.nio.ByteBuffer;
 import java.nio.charset.CharacterCodingException;
@@ -38,10 +39,6 @@
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.CurrentLocationColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.FutureLocationColumnFamily;
 import org.apache.hadoop.io.Text;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.gson.Gson;
 
 /**
  * This class is used to serialize and deserialize root tablet metadata using GSon. The only data
@@ -51,7 +48,6 @@
  */
 public class RootTabletMetadata {
 
-  private static final Logger log = LoggerFactory.getLogger(RootTabletMetadata.class);
   private static final CharsetDecoder UTF8_error_detecting_decoder = UTF_8.newDecoder();
   private static final Predicate<Entry<String,TreeMap<String,String>>> isLocationCF = e -> {
     String fam = e.getKey();
@@ -59,7 +55,6 @@
         || fam.equals(FutureLocationColumnFamily.STR_NAME);
   };
 
-  // JSON Mapping Version 1. Released with Accumulo version 2.1.0
   private static final int VERSION = 1;
 
   // This class is used to serialize and deserialize root tablet metadata using GSon. Any changes to
@@ -74,10 +69,20 @@
      */
     private final TreeMap<String,TreeMap<String,String>> columnValues;
 
-    public Data(int version, TreeMap<String,TreeMap<String,String>> columnValues) {
+    private Data(int version, TreeMap<String,TreeMap<String,String>> columnValues) {
       this.version = version;
       this.columnValues = columnValues;
     }
+
+    public int getVersion() {
+      return version;
+    }
+
+    public static boolean needsUpgrade(final String json) {
+      var rootData = GSON.get().fromJson(json, Data.class);
+      int currVersion = rootData.getVersion();
+      return currVersion < VERSION;
+    }
   }
 
   /**
@@ -93,12 +98,14 @@
     }
   }
 
-  private final Gson gson = new Gson();
   private final Data data;
 
   public RootTabletMetadata(String json) {
-    log.trace("Creating root tablet metadata from stored JSON: {}", json);
-    this.data = gson.fromJson(json, Data.class);
+    this(GSON.get().fromJson(json, Data.class));
+  }
+
+  private RootTabletMetadata(final Data data) {
+    this.data = data;
     checkArgument(data.version == VERSION, "Invalid Root Table Metadata JSON version %s",
         data.version);
     data.columnValues.forEach((fam, qualVals) -> {
@@ -108,7 +115,7 @@
   }
 
   public RootTabletMetadata() {
-    this.data = new Data(VERSION, new TreeMap<>());
+    data = new Data(VERSION, new TreeMap<>());
   }
 
   /**
@@ -145,26 +152,33 @@
     }
   }
 
-  /**
-   * Convert this class to a {@link TabletMetadata}
-   */
-  public TabletMetadata toTabletMetadata() {
+  public Stream<SimpleImmutableEntry<Key,Value>> toKeyValues() {
     String row = RootTable.EXTENT.toMetaRow().toString();
-    // use a stream so we don't have to re-sort in a new TreeMap<Key,Value> structure
-    Stream<SimpleImmutableEntry<Key,Value>> entries = data.columnValues.entrySet().stream()
+    return data.columnValues.entrySet().stream()
         .flatMap(famToQualVal -> famToQualVal.getValue().entrySet().stream()
             .map(qualVal -> new SimpleImmutableEntry<>(
                 new Key(row, famToQualVal.getKey(), qualVal.getKey(), 1),
                 new Value(qualVal.getValue()))));
-    return TabletMetadata.convertRow(entries.iterator(),
+  }
+
+  /**
+   * Convert this class to a {@link TabletMetadata}
+   */
+  public TabletMetadata toTabletMetadata() {
+    // use a stream so we don't have to re-sort in a new TreeMap<Key,Value> structure
+    return TabletMetadata.convertRow(toKeyValues().iterator(),
         EnumSet.allOf(TabletMetadata.ColumnType.class), false);
   }
 
+  public static boolean needsUpgrade(final String json) {
+    return Data.needsUpgrade(json);
+  }
+
   /**
    * @return a JSON representation of the root tablet's data.
    */
   public String toJson() {
-    return gson.toJson(data);
+    return GSON.get().toJson(data);
   }
 
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletMetadata.java b/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletMetadata.java
index c1e8161..33d307c 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletMetadata.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletMetadata.java
@@ -18,7 +18,6 @@
  */
 package org.apache.accumulo.core.metadata.schema;
 
-import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily.COMPACT_QUAL;
 import static org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily.DIRECTORY_QUAL;
 import static org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily.FLUSH_QUAL;
@@ -38,7 +37,6 @@
 import java.util.Optional;
 import java.util.OptionalLong;
 import java.util.Set;
-import java.util.SortedMap;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.clientImpl.ClientContext;
@@ -47,16 +45,15 @@
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock;
 import org.apache.accumulo.core.fate.zookeeper.ZooCache;
+import org.apache.accumulo.core.lock.ServiceLock;
+import org.apache.accumulo.core.lock.ServiceLockData;
 import org.apache.accumulo.core.metadata.StoredTabletFile;
 import org.apache.accumulo.core.metadata.SuspendingTServer;
 import org.apache.accumulo.core.metadata.TServerInstance;
-import org.apache.accumulo.core.metadata.TabletFile;
 import org.apache.accumulo.core.metadata.TabletLocationState;
 import org.apache.accumulo.core.metadata.TabletState;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.BulkFileColumnFamily;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ChoppedColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ClonedColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.CurrentLocationColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
@@ -64,50 +61,81 @@
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.FutureLocationColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LastLocationColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LogColumnFamily;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.MergedColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ScanFileColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.SuspendLocationColumn;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.TabletColumnFamily;
 import org.apache.accumulo.core.tabletserver.log.LogEntry;
-import org.apache.accumulo.core.util.HostAndPort;
-import org.apache.accumulo.core.util.ServerServices;
 import org.apache.hadoop.io.Text;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
+import com.google.common.base.Supplier;
+import com.google.common.base.Suppliers;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.ImmutableSortedMap;
+import com.google.common.collect.ImmutableSortedMap.Builder;
+import com.google.common.net.HostAndPort;
 
 public class TabletMetadata {
   private static final Logger log = LoggerFactory.getLogger(TabletMetadata.class);
 
-  private TableId tableId;
-  private Text prevEndRow;
-  private boolean sawPrevEndRow = false;
-  private Text oldPrevEndRow;
-  private boolean sawOldPrevEndRow = false;
-  private Text endRow;
-  private Location location;
-  private Map<StoredTabletFile,DataFileValue> files;
-  private List<StoredTabletFile> scans;
-  private Map<TabletFile,Long> loadedFiles;
-  private EnumSet<ColumnType> fetchedCols;
-  private KeyExtent extent;
-  private Location last;
-  private SuspendingTServer suspend;
-  private String dirName;
-  private MetadataTime time;
-  private String cloned;
-  private SortedMap<Key,Value> keyValues;
-  private OptionalLong flush = OptionalLong.empty();
-  private List<LogEntry> logs;
-  private OptionalLong compact = OptionalLong.empty();
-  private Double splitRatio = null;
-  private Map<ExternalCompactionId,ExternalCompactionMetadata> extCompactions;
-  private boolean chopped = false;
+  private final TableId tableId;
+  private final Text prevEndRow;
+  private final boolean sawPrevEndRow;
+  private final Text oldPrevEndRow;
+  private final boolean sawOldPrevEndRow;
+  private final Text endRow;
+  private final Location location;
+  private final Map<StoredTabletFile,DataFileValue> files;
+  private final List<StoredTabletFile> scans;
+  private final Map<StoredTabletFile,Long> loadedFiles;
+  private final EnumSet<ColumnType> fetchedCols;
+  private final Supplier<KeyExtent> extent;
+  private final Location last;
+  private final SuspendingTServer suspend;
+  private final String dirName;
+  private final MetadataTime time;
+  private final String cloned;
+  private final List<Entry<Key,Value>> keyValues;
+  private final OptionalLong flush;
+  private final List<LogEntry> logs;
+  private final OptionalLong compact;
+  private final Double splitRatio;
+  private final Map<ExternalCompactionId,ExternalCompactionMetadata> extCompactions;
+  private final boolean merged;
+
+  private TabletMetadata(Builder tmBuilder) {
+    this.tableId = tmBuilder.tableId;
+    this.prevEndRow = tmBuilder.prevEndRow;
+    this.sawPrevEndRow = tmBuilder.sawPrevEndRow;
+    this.oldPrevEndRow = tmBuilder.oldPrevEndRow;
+    this.sawOldPrevEndRow = tmBuilder.sawOldPrevEndRow;
+    this.endRow = tmBuilder.endRow;
+    this.location = tmBuilder.location;
+    this.files = Objects.requireNonNull(tmBuilder.files.build());
+    this.scans = Objects.requireNonNull(tmBuilder.scans.build());
+    this.loadedFiles = tmBuilder.loadedFiles.build();
+    this.fetchedCols = Objects.requireNonNull(tmBuilder.fetchedCols);
+    this.last = tmBuilder.last;
+    this.suspend = tmBuilder.suspend;
+    this.dirName = tmBuilder.dirName;
+    this.time = tmBuilder.time;
+    this.cloned = tmBuilder.cloned;
+    this.keyValues =
+        Optional.ofNullable(tmBuilder.keyValues).map(ImmutableList.Builder::build).orElse(null);
+    this.flush = tmBuilder.flush;
+    this.logs = Objects.requireNonNull(tmBuilder.logs.build());
+    this.compact = Objects.requireNonNull(tmBuilder.compact);
+    this.splitRatio = tmBuilder.splitRatio;
+    this.extCompactions = Objects.requireNonNull(tmBuilder.extCompactions.build());
+    this.merged = tmBuilder.merged;
+    this.extent =
+        Suppliers.memoize(() -> new KeyExtent(getTableId(), getEndRow(), getPrevEndRow()));
+  }
 
   public enum LocationType {
     CURRENT, FUTURE, LAST
@@ -129,8 +157,8 @@
     COMPACT_ID,
     SPLIT_RATIO,
     SUSPEND,
-    CHOPPED,
-    ECOMP
+    ECOMP,
+    MERGED
   }
 
   public static class Location {
@@ -233,10 +261,7 @@
   }
 
   public KeyExtent getExtent() {
-    if (extent == null) {
-      extent = new KeyExtent(getTableId(), getEndRow(), getPrevEndRow());
-    }
-    return extent;
+    return extent.get();
   }
 
   private void ensureFetched(ColumnType col) {
@@ -285,7 +310,7 @@
     return location != null && location.getType() == LocationType.CURRENT;
   }
 
-  public Map<TabletFile,Long> getLoaded() {
+  public Map<StoredTabletFile,Long> getLoaded() {
     ensureFetched(ColumnType.LOADED);
     return loadedFiles;
   }
@@ -350,12 +375,12 @@
     return splitRatio;
   }
 
-  public boolean hasChopped() {
-    ensureFetched(ColumnType.CHOPPED);
-    return chopped;
+  public boolean hasMerged() {
+    ensureFetched(ColumnType.MERGED);
+    return merged;
   }
 
-  public SortedMap<Key,Value> getKeyValues() {
+  public List<Entry<Key,Value>> getKeyValues() {
     Preconditions.checkState(keyValues != null, "Requested key values when it was not saved");
     return keyValues;
   }
@@ -375,7 +400,7 @@
       // only care about the state so don't need walogs and chopped params
       // Use getExtent() when passing the extent as the private reference may not have been
       // initialized yet. This will also ensure PREV_ROW was fetched
-      var tls = new TabletLocationState(getExtent(), future, current, last, suspend, null, false);
+      var tls = new TabletLocationState(getExtent(), future, current, last, suspend, null);
       return tls.getState(liveTServers);
     } catch (TabletLocationState.BadLocationStateException blse) {
       throw new IllegalArgumentException("Error creating TabletLocationState", blse);
@@ -392,16 +417,7 @@
       EnumSet<ColumnType> fetchedColumns, boolean buildKeyValueMap) {
     Objects.requireNonNull(rowIter);
 
-    TabletMetadata te = new TabletMetadata();
-    final ImmutableSortedMap.Builder<Key,Value> kvBuilder =
-        buildKeyValueMap ? ImmutableSortedMap.naturalOrder() : null;
-
-    final var filesBuilder = ImmutableMap.<StoredTabletFile,DataFileValue>builder();
-    final var scansBuilder = ImmutableList.<StoredTabletFile>builder();
-    final var logsBuilder = ImmutableList.<LogEntry>builder();
-    final var extCompBuilder =
-        ImmutableMap.<ExternalCompactionId,ExternalCompactionMetadata>builder();
-    final var loadedFilesBuilder = ImmutableMap.<TabletFile,Long>builder();
+    final var tmBuilder = new Builder();
     ByteSequence row = null;
 
     while (rowIter.hasNext()) {
@@ -412,14 +428,14 @@
       final String qual = key.getColumnQualifierData().toString();
 
       if (buildKeyValueMap) {
-        kvBuilder.put(key, kv.getValue());
+        tmBuilder.keyValue(kv);
       }
 
       if (row == null) {
         row = key.getRowData();
         KeyExtent ke = KeyExtent.fromMetaRow(key.getRow());
-        te.endRow = ke.endRow();
-        te.tableId = ke.tableId();
+        tmBuilder.endRow(ke.endRow());
+        tmBuilder.table(ke.tableId());
       } else if (!row.equals(key.getRowData())) {
         throw new IllegalArgumentException(
             "Input contains more than one row : " + row + " " + key.getRowData());
@@ -429,15 +445,15 @@
         case TabletColumnFamily.STR_NAME:
           switch (qual) {
             case PREV_ROW_QUAL:
-              te.prevEndRow = TabletColumnFamily.decodePrevEndRow(kv.getValue());
-              te.sawPrevEndRow = true;
+              tmBuilder.prevEndRow(TabletColumnFamily.decodePrevEndRow(kv.getValue()));
+              tmBuilder.sawPrevEndRow(true);
               break;
             case OLD_PREV_ROW_QUAL:
-              te.oldPrevEndRow = TabletColumnFamily.decodePrevEndRow(kv.getValue());
-              te.sawOldPrevEndRow = true;
+              tmBuilder.oldPrevEndRow(TabletColumnFamily.decodePrevEndRow(kv.getValue()));
+              tmBuilder.sawOldPrevEndRow(true);
               break;
             case SPLIT_RATIO_QUAL:
-              te.splitRatio = Double.parseDouble(val);
+              tmBuilder.splitRatio(Double.parseDouble(val));
               break;
           }
           break;
@@ -446,88 +462,70 @@
             case DIRECTORY_QUAL:
               Preconditions.checkArgument(ServerColumnFamily.isValidDirCol(val),
                   "Saw invalid dir name %s %s", key, val);
-              te.dirName = val;
+              tmBuilder.dirName(val);
               break;
             case TIME_QUAL:
-              te.time = MetadataTime.parse(val);
+              tmBuilder.time(MetadataTime.parse(val));
               break;
             case FLUSH_QUAL:
-              te.flush = OptionalLong.of(Long.parseLong(val));
+              tmBuilder.flush(Long.parseLong(val));
               break;
             case COMPACT_QUAL:
-              te.compact = OptionalLong.of(Long.parseLong(val));
+              tmBuilder.compact(Long.parseLong(val));
               break;
           }
           break;
         case DataFileColumnFamily.STR_NAME:
-          filesBuilder.put(new StoredTabletFile(qual), new DataFileValue(val));
+          tmBuilder.file(new StoredTabletFile(qual), new DataFileValue(val));
           break;
         case BulkFileColumnFamily.STR_NAME:
-          loadedFilesBuilder.put(new StoredTabletFile(qual),
+          tmBuilder.loadedFile(new StoredTabletFile(qual),
               BulkFileColumnFamily.getBulkLoadTid(val));
           break;
         case CurrentLocationColumnFamily.STR_NAME:
-          te.setLocationOnce(val, qual, LocationType.CURRENT);
+          tmBuilder.location(val, qual, LocationType.CURRENT);
           break;
         case FutureLocationColumnFamily.STR_NAME:
-          te.setLocationOnce(val, qual, LocationType.FUTURE);
+          tmBuilder.location(val, qual, LocationType.FUTURE);
           break;
         case LastLocationColumnFamily.STR_NAME:
-          te.last = Location.last(val, qual);
+          tmBuilder.last(Location.last(val, qual));
           break;
         case SuspendLocationColumn.STR_NAME:
-          te.suspend = SuspendingTServer.fromValue(kv.getValue());
+          tmBuilder.suspend(SuspendingTServer.fromValue(kv.getValue()));
           break;
         case ScanFileColumnFamily.STR_NAME:
-          scansBuilder.add(new StoredTabletFile(qual));
+          tmBuilder.scan(new StoredTabletFile(qual));
           break;
         case ClonedColumnFamily.STR_NAME:
-          te.cloned = val;
+          tmBuilder.cloned(val);
           break;
         case LogColumnFamily.STR_NAME:
-          logsBuilder.add(LogEntry.fromMetaWalEntry(kv));
+          tmBuilder.log(LogEntry.fromMetaWalEntry(kv));
           break;
         case ExternalCompactionColumnFamily.STR_NAME:
-          extCompBuilder.put(ExternalCompactionId.of(qual),
+          tmBuilder.extCompaction(ExternalCompactionId.of(qual),
               ExternalCompactionMetadata.fromJson(val));
           break;
-        case ChoppedColumnFamily.STR_NAME:
-          te.chopped = true;
+        case MergedColumnFamily.STR_NAME:
+          tmBuilder.merged(true);
           break;
         default:
           throw new IllegalStateException("Unexpected family " + fam);
       }
     }
 
-    te.files = filesBuilder.build();
-    te.loadedFiles = loadedFilesBuilder.build();
-    te.fetchedCols = fetchedColumns;
-    te.scans = scansBuilder.build();
-    te.logs = logsBuilder.build();
-    te.extCompactions = extCompBuilder.build();
-    if (buildKeyValueMap) {
-      te.keyValues = kvBuilder.build();
-    }
-    return te;
-  }
-
-  private void setLocationOnce(String val, String qual, LocationType lt) {
-    if (location != null) {
-      throw new IllegalStateException("Attempted to set second location for tableId: " + tableId
-          + " endrow: " + endRow + " -- " + location + " " + qual + " " + val);
-    }
-    location = new Location(val, qual, lt);
+    return tmBuilder.build(fetchedColumns);
   }
 
   @VisibleForTesting
   static TabletMetadata create(String id, String prevEndRow, String endRow) {
-    TabletMetadata te = new TabletMetadata();
-    te.tableId = TableId.of(id);
-    te.sawPrevEndRow = true;
-    te.prevEndRow = prevEndRow == null ? null : new Text(prevEndRow);
-    te.endRow = endRow == null ? null : new Text(endRow);
-    te.fetchedCols = EnumSet.of(ColumnType.PREV_ROW);
-    return te;
+    final var tmBuilder = new Builder();
+    tmBuilder.table(TableId.of(id));
+    tmBuilder.sawPrevEndRow(true);
+    tmBuilder.prevEndRow(prevEndRow == null ? null : new Text(prevEndRow));
+    tmBuilder.endRow(endRow == null ? null : new Text(endRow));
+    return tmBuilder.build(EnumSet.of(ColumnType.PREV_ROW));
   }
 
   /**
@@ -553,17 +551,139 @@
    */
   private static Optional<TServerInstance> checkServer(ClientContext context, String path,
       String zPath) {
-    Optional<TServerInstance> server = Optional.empty();
     final var lockPath = ServiceLock.path(path + "/" + zPath);
     ZooCache.ZcStat stat = new ZooCache.ZcStat();
-    byte[] lockData = ServiceLock.getLockData(context.getZooCache(), lockPath, stat);
-
     log.trace("Checking server at ZK path = " + lockPath);
-    if (lockData != null) {
-      ServerServices services = new ServerServices(new String(lockData, UTF_8));
-      HostAndPort client = services.getAddress(ServerServices.Service.TSERV_CLIENT);
-      server = Optional.of(new TServerInstance(client, stat.getEphemeralOwner()));
+    return ServiceLock.getLockData(context.getZooCache(), lockPath, stat)
+        .map(sld -> sld.getAddress(ServiceLockData.ThriftService.TSERV))
+        .map(address -> new TServerInstance(address, stat.getEphemeralOwner()));
+  }
+
+  static class Builder {
+    private TableId tableId;
+    private Text prevEndRow;
+    private boolean sawPrevEndRow;
+    private Text oldPrevEndRow;
+    private boolean sawOldPrevEndRow;
+    private Text endRow;
+    private Location location;
+    private final ImmutableMap.Builder<StoredTabletFile,DataFileValue> files =
+        ImmutableMap.builder();
+    private final ImmutableList.Builder<StoredTabletFile> scans = ImmutableList.builder();
+    private final ImmutableMap.Builder<StoredTabletFile,Long> loadedFiles = ImmutableMap.builder();
+    private EnumSet<ColumnType> fetchedCols;
+    private Location last;
+    private SuspendingTServer suspend;
+    private String dirName;
+    private MetadataTime time;
+    private String cloned;
+    private ImmutableList.Builder<Entry<Key,Value>> keyValues;
+    private OptionalLong flush = OptionalLong.empty();
+    private final ImmutableList.Builder<LogEntry> logs = ImmutableList.builder();
+    private OptionalLong compact = OptionalLong.empty();
+    private Double splitRatio = null;
+    private final ImmutableMap.Builder<ExternalCompactionId,
+        ExternalCompactionMetadata> extCompactions = ImmutableMap.builder();
+    private boolean merged;
+
+    void table(TableId tableId) {
+      this.tableId = tableId;
     }
-    return server;
+
+    void endRow(Text endRow) {
+      this.endRow = endRow;
+    }
+
+    void prevEndRow(Text prevEndRow) {
+      this.prevEndRow = prevEndRow;
+    }
+
+    void sawPrevEndRow(boolean sawPrevEndRow) {
+      this.sawPrevEndRow = sawPrevEndRow;
+    }
+
+    void oldPrevEndRow(Text oldPrevEndRow) {
+      this.oldPrevEndRow = oldPrevEndRow;
+    }
+
+    void sawOldPrevEndRow(boolean sawOldPrevEndRow) {
+      this.sawOldPrevEndRow = sawOldPrevEndRow;
+    }
+
+    void splitRatio(Double splitRatio) {
+      this.splitRatio = splitRatio;
+    }
+
+    void dirName(String dirName) {
+      this.dirName = dirName;
+    }
+
+    void time(MetadataTime time) {
+      this.time = time;
+    }
+
+    void flush(long flush) {
+      this.flush = OptionalLong.of(flush);
+    }
+
+    void compact(long compact) {
+      this.compact = OptionalLong.of(compact);
+    }
+
+    void file(StoredTabletFile stf, DataFileValue dfv) {
+      this.files.put(stf, dfv);
+    }
+
+    void loadedFile(StoredTabletFile stf, Long tid) {
+      this.loadedFiles.put(stf, tid);
+    }
+
+    void location(String val, String qual, LocationType lt) {
+      if (location != null) {
+        throw new IllegalStateException("Attempted to set second location for tableId: " + tableId
+            + " endrow: " + endRow + " -- " + location + " " + qual + " " + val);
+      }
+      this.location = new Location(val, qual, lt);
+    }
+
+    void last(Location last) {
+      this.last = last;
+    }
+
+    void suspend(SuspendingTServer suspend) {
+      this.suspend = suspend;
+    }
+
+    void scan(StoredTabletFile stf) {
+      this.scans.add(stf);
+    }
+
+    void cloned(String cloned) {
+      this.cloned = cloned;
+    }
+
+    void log(LogEntry log) {
+      this.logs.add(log);
+    }
+
+    void extCompaction(ExternalCompactionId id, ExternalCompactionMetadata metadata) {
+      this.extCompactions.put(id, metadata);
+    }
+
+    void merged(boolean merged) {
+      this.merged = merged;
+    }
+
+    void keyValue(Entry<Key,Value> kv) {
+      if (this.keyValues == null) {
+        this.keyValues = ImmutableList.builder();
+      }
+      this.keyValues.add(kv);
+    }
+
+    TabletMetadata build(EnumSet<ColumnType> fetchedCols) {
+      this.fetchedCols = fetchedCols;
+      return new TabletMetadata(this);
+    }
   }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletsMetadata.java b/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletsMetadata.java
index 6ce050c..95045a2 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletsMetadata.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletsMetadata.java
@@ -34,13 +34,17 @@
 import java.util.Collection;
 import java.util.Collections;
 import java.util.EnumSet;
+import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.NoSuchElementException;
 import java.util.Objects;
+import java.util.Optional;
 import java.util.Set;
+import java.util.function.Consumer;
 import java.util.function.Function;
+import java.util.stream.Collectors;
 import java.util.stream.Stream;
 import java.util.stream.StreamSupport;
 
@@ -59,7 +63,7 @@
 import org.apache.accumulo.core.fate.zookeeper.ZooCache;
 import org.apache.accumulo.core.fate.zookeeper.ZooReader;
 import org.apache.accumulo.core.iterators.user.WholeRowIterator;
-import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.metadata.RootTable;
 import org.apache.accumulo.core.metadata.schema.Ample.DataLevel;
 import org.apache.accumulo.core.metadata.schema.Ample.ReadConsistency;
@@ -72,6 +76,7 @@
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.FutureLocationColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LastLocationColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LogColumnFamily;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.MergedColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ScanFileColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.SuspendLocationColumn;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType;
@@ -94,6 +99,8 @@
     private final List<Text> families = new ArrayList<>();
     private final List<ColumnFQ> qualifiers = new ArrayList<>();
     private Set<KeyExtent> extentsToFetch = null;
+    private boolean fetchTablets = false;
+    private Optional<Consumer<KeyExtent>> notFoundHandler;
     private Ample.DataLevel level;
     private String table;
     private Range range;
@@ -104,7 +111,6 @@
     private TableId tableId;
     private ReadConsistency readConsistency = ReadConsistency.IMMEDIATE;
     private final AccumuloClient _client;
-    private Collection<KeyExtent> extents = null;
 
     Builder(AccumuloClient client) {
       this._client = client;
@@ -112,7 +118,7 @@
 
     @Override
     public TabletsMetadata build() {
-      if (extents != null) {
+      if (fetchTablets) {
         // setting multiple extents with forTablets(extents) is mutually exclusive with these
         // single-tablet options
         checkState(range == null && table == null && level == null && !checkConsistency);
@@ -120,7 +126,8 @@
       }
 
       checkState((level == null) != (table == null),
-          "scanTable() cannot be used in conjunction with forLevel(), forTable() or forTablet()");
+          "scanTable() cannot be used in conjunction with forLevel(), forTable() or forTablet() %s %s",
+          level, table);
       if (level == DataLevel.ROOT) {
         ClientContext ctx = ((ClientContext) _client);
         return new TabletsMetadata(getRootMetadata(ctx, readConsistency));
@@ -132,7 +139,7 @@
     private TabletsMetadata buildExtents(AccumuloClient client) {
 
       Map<DataLevel,List<KeyExtent>> groupedExtents =
-          extents.stream().collect(groupingBy(ke -> DataLevel.of(ke.tableId())));
+          extentsToFetch.stream().collect(groupingBy(ke -> DataLevel.of(ke.tableId())));
 
       List<Iterable<TabletMetadata>> iterables = new ArrayList<>();
 
@@ -175,19 +182,39 @@
             closables.add(scanner);
 
           } catch (TableNotFoundException e) {
-            throw new RuntimeException(e);
+            throw new IllegalStateException(e);
           }
 
         }
       }
 
-      return new TabletsMetadata(() -> {
+      if (notFoundHandler.isPresent()) {
+        HashSet<KeyExtent> extentsNotSeen = new HashSet<>(extentsToFetch);
+
+        var tablets = iterables.stream().flatMap(i -> StreamSupport.stream(i.spliterator(), false))
+            .filter(tabletMetadata -> extentsNotSeen.remove(tabletMetadata.getExtent()))
+            .collect(Collectors.toList());
+
+        extentsNotSeen.forEach(notFoundHandler.orElseThrow());
+
         for (AutoCloseable closable : closables) {
-          closable.close();
+          try {
+            closable.close();
+          } catch (Exception e) {
+            throw new RuntimeException(e);
+          }
         }
-      }, () -> iterables.stream().flatMap(i -> StreamSupport.stream(i.spliterator(), false))
-          .filter(tabletMetadata -> extentsToFetch.contains(tabletMetadata.getExtent()))
-          .iterator());
+
+        return new TabletsMetadata(() -> {}, tablets);
+      } else {
+        return new TabletsMetadata(() -> {
+          for (AutoCloseable closable : closables) {
+            closable.close();
+          }
+        }, () -> iterables.stream().flatMap(i -> StreamSupport.stream(i.spliterator(), false))
+            .filter(tabletMetadata -> extentsToFetch.contains(tabletMetadata.getExtent()))
+            .iterator());
+      }
 
     }
 
@@ -239,7 +266,7 @@
           return new TabletsMetadata(scanner, tmi);
         }
       } catch (TableNotFoundException e) {
-        throw new RuntimeException(e);
+        throw new IllegalStateException(e);
       }
     }
 
@@ -253,7 +280,7 @@
 
     @Override
     public Options checkConsistency() {
-      checkState(extents == null, "Unable to check consistency of non-contiguous tablets");
+      checkState(!fetchTablets, "Unable to check consistency of non-contiguous tablets");
       this.checkConsistency = true;
       return this;
     }
@@ -310,6 +337,9 @@
           case ECOMP:
             families.add(ExternalCompactionColumnFamily.NAME);
             break;
+          case MERGED:
+            families.add(MergedColumnFamily.NAME);
+            break;
           default:
             throw new IllegalArgumentException("Unknown col type " + colToFetch);
 
@@ -356,10 +386,12 @@
     }
 
     @Override
-    public Options forTablets(Collection<KeyExtent> extents) {
+    public Options forTablets(Collection<KeyExtent> extents,
+        Optional<Consumer<KeyExtent>> notFoundHandler) {
       this.level = null;
-      this.extents = extents;
       this.extentsToFetch = Set.copyOf(extents);
+      this.notFoundHandler = Objects.requireNonNull(notFoundHandler);
+      this.fetchTablets = true;
       return this;
     }
 
@@ -412,6 +444,8 @@
 
     /**
      * Checks that the metadata table forms a linked list and automatically backs up until it does.
+     * May cause {@link TabletDeletedException} to be thrown while reading tablets metadata in the
+     * case where a table is deleted or merge runs concurrently with scan.
      */
     Options checkConsistency();
 
@@ -449,8 +483,15 @@
     /**
      * Get the tablet metadata for the given extents. This will only return tablets where the end
      * row and prev end row exactly match the given extents.
+     *
+     * @param notFoundConsumer if a consumer is present, the extents that do not exists in the
+     *        metadata store are passed to the consumer. If the missing extents are not needed, then
+     *        pass Optional.empty() and it will be more efficient. Computing the missing extents
+     *        requires buffering all tablet metadata in memory before returning anything, when
+     *        Optional.empty() is passed this buffering is not done.
      */
-    Options forTablets(Collection<KeyExtent> extents);
+    Options forTablets(Collection<KeyExtent> extents,
+        Optional<Consumer<KeyExtent>> notFoundConsumer);
 
     /**
      * This method automatically determines where the metadata for the passed in table ID resides.
@@ -465,7 +506,7 @@
      * {@link TabletsSection#getRange()}
      */
     default RangeOptions scanMetadataTable() {
-      return scanTable(MetadataTable.NAME);
+      return scanTable(AccumuloTable.METADATA.tableName());
     }
 
     /**
@@ -557,7 +598,7 @@
           byte[] bytes = zooReader.getData(path);
           return new RootTabletMetadata(new String(bytes, UTF_8)).toTabletMetadata();
         } catch (InterruptedException | KeeperException e) {
-          throw new RuntimeException(e);
+          throw new IllegalStateException(e);
         }
       default:
         throw new IllegalArgumentException("Unknown consistency level " + readConsistency);
@@ -592,7 +633,7 @@
         // avoid wrapping runtime w/ runtime
         throw e;
       } catch (Exception e) {
-        throw new RuntimeException(e);
+        throw new IllegalStateException(e);
       }
     }
   }
diff --git a/core/src/main/java/org/apache/accumulo/core/metrics/MetricsInfo.java b/core/src/main/java/org/apache/accumulo/core/metrics/MetricsInfo.java
index 4277e11..b6ef72b 100644
--- a/core/src/main/java/org/apache/accumulo/core/metrics/MetricsInfo.java
+++ b/core/src/main/java/org/apache/accumulo/core/metrics/MetricsInfo.java
@@ -24,7 +24,7 @@
 import java.util.List;
 import java.util.Objects;
 
-import org.apache.accumulo.core.util.HostAndPort;
+import com.google.common.net.HostAndPort;
 
 import io.micrometer.core.instrument.MeterRegistry;
 import io.micrometer.core.instrument.Tag;
diff --git a/core/src/main/java/org/apache/accumulo/core/metrics/MetricsProducer.java b/core/src/main/java/org/apache/accumulo/core/metrics/MetricsProducer.java
index 8cf2ffc..ee6eb6e 100644
--- a/core/src/main/java/org/apache/accumulo/core/metrics/MetricsProducer.java
+++ b/core/src/main/java/org/apache/accumulo/core/metrics/MetricsProducer.java
@@ -55,6 +55,13 @@
  * <tr>
  * <td>N/A</td>
  * <td>N/A</td>
+ * <td>{@value #METRICS_LOW_MEMORY}</td>
+ * <td>Guage</td>
+ * <td>reports 1 when process memory usage is above threshold, 0 when memory is okay</td>
+ * </tr>
+ * <tr>
+ * <td>N/A</td>
+ * <td>N/A</td>
  * <td>{@value #METRICS_COMPACTOR_MAJC_STUCK}</td>
  * <td>LongTaskTimer</td>
  * <td></td>
@@ -480,6 +487,13 @@
  * <td>The compaction service information is in a tag:
  * id={i|e}_{compactionServiceName}_{executor_name}</td>
  * </tr>
+ * <tr>
+ * <td></td>
+ * <td></td>
+ * <td>{@link #METRICS_MAJC_PAUSED}</td>
+ * <td>Counter</td>
+ * <td></td>
+ * </tr>
  * <!-- minor compactions -->
  * <tr>
  * <td>Queue</td>
@@ -495,40 +509,11 @@
  * <td>Timer</td>
  * <td></td>
  * </tr>
- * <!-- replication -->
  * <tr>
- * <td>ReplicationQueue</td>
- * <td>Stat</td>
- * <td>{@value #METRICS_REPLICATION_QUEUE}</td>
- * <td>Timer</td>
  * <td></td>
- * </tr>
- * <tr>
- * <td>ReplicationQueue10m</td>
- * <td>Quantiles</td>
- * <td>N/A</td>
- * <td>N/A</td>
  * <td></td>
- * </tr>
- * <tr>
- * <td>filesPendingReplication</td>
- * <td>Stat</td>
- * <td>{@value #METRICS_REPLICATION_PENDING_FILES}</td>
- * <td>Gauge</td>
- * <td></td>
- * </tr>
- * <tr>
- * <td>maxReplicationThreads</td>
- * <td>Stat</td>
- * <td>{@value #METRICS_REPLICATION_THREADS}</td>
- * <td>Gauge</td>
- * <td></td>
- * </tr>
- * <tr>
- * <td>numPeers</td>
- * <td>Stat</td>
- * <td>{@value #METRICS_REPLICATION_PEERS}</td>
- * <td>Gauge</td>
+ * <td>{@value #METRICS_MINC_PAUSED}</td>
+ * <td>Counter</td>
  * <td></td>
  * </tr>
  * <!-- Updates (ingest) -->
@@ -604,6 +589,7 @@
 
   Logger LOG = LoggerFactory.getLogger(MetricsProducer.class);
 
+  String METRICS_LOW_MEMORY = "accumulo.detected.low.memory";
   String METRICS_SERVER_IDLE = "accumulo.server.idle";
 
   String METRICS_COMPACTOR_PREFIX = "accumulo.compactor.";
@@ -637,16 +623,12 @@
   String METRICS_MAJC_PREFIX = "accumulo.tserver.compactions.majc.";
   String METRICS_MAJC_QUEUED = METRICS_MAJC_PREFIX + "queued";
   String METRICS_MAJC_RUNNING = METRICS_MAJC_PREFIX + "running";
+  String METRICS_MAJC_PAUSED = METRICS_MAJC_PREFIX + "paused";
 
   String METRICS_MINC_PREFIX = "accumulo.tserver.compactions.minc.";
   String METRICS_MINC_QUEUED = METRICS_MINC_PREFIX + "queued";
   String METRICS_MINC_RUNNING = METRICS_MINC_PREFIX + "running";
-
-  String METRICS_REPLICATION_PREFIX = "accumulo.replication.";
-  String METRICS_REPLICATION_QUEUE = METRICS_REPLICATION_PREFIX + "queue";
-  String METRICS_REPLICATION_PENDING_FILES = METRICS_REPLICATION_PREFIX + "files.pending";
-  String METRICS_REPLICATION_PEERS = METRICS_REPLICATION_PREFIX + "peers";
-  String METRICS_REPLICATION_THREADS = METRICS_REPLICATION_PREFIX + "threads";
+  String METRICS_MINC_PAUSED = METRICS_MINC_PREFIX + "paused";
 
   String METRICS_SCAN_PREFIX = "accumulo.scan.";
   String METRICS_SCAN_TIMES = METRICS_SCAN_PREFIX + "times";
@@ -666,6 +648,8 @@
   String METRICS_SCAN_QUERY_SCAN_RESULTS = METRICS_SCAN_PREFIX + "query.results";
   String METRICS_SCAN_QUERY_SCAN_RESULTS_BYTES = METRICS_SCAN_PREFIX + "query.results.bytes";
   String METRICS_SCAN_SCANNED_ENTRIES = METRICS_SCAN_PREFIX + "query.scanned.entries";
+  String METRICS_SCAN_PAUSED_FOR_MEM = METRICS_SCAN_PREFIX + "paused.for.memory";
+  String METRICS_SCAN_RETURN_FOR_MEM = METRICS_SCAN_PREFIX + "return.early.for.memory";
 
   String METRICS_SCAN_TABLET_METADATA_CACHE = METRICS_SCAN_PREFIX + "tablet.metadata.cache";
 
@@ -702,11 +686,15 @@
   String METRICS_BLOCKCACHE_PREFIX = "accumulo.blockcache.";
   String METRICS_BLOCKCACHE_INDEX_HITCOUNT = METRICS_BLOCKCACHE_PREFIX + "index.hitcount";
   String METRICS_BLOCKCACHE_INDEX_REQUESTCOUNT = METRICS_BLOCKCACHE_PREFIX + "index.requestcount";
+  String METRICS_BLOCKCACHE_INDEX_EVICTIONCOUNT = METRICS_BLOCKCACHE_PREFIX + "index.evictioncount";
   String METRICS_BLOCKCACHE_DATA_HITCOUNT = METRICS_BLOCKCACHE_PREFIX + "data.hitcount";
   String METRICS_BLOCKCACHE_DATA_REQUESTCOUNT = METRICS_BLOCKCACHE_PREFIX + "data.requestcount";
+  String METRICS_BLOCKCACHE_DATA_EVICTIONCOUNT = METRICS_BLOCKCACHE_PREFIX + "data.evictioncount";
   String METRICS_BLOCKCACHE_SUMMARY_HITCOUNT = METRICS_BLOCKCACHE_PREFIX + "summary.hitcount";
   String METRICS_BLOCKCACHE_SUMMARY_REQUESTCOUNT =
       METRICS_BLOCKCACHE_PREFIX + "summary.requestcount";
+  String METRICS_BLOCKCACHE_SUMMARY_EVICTIONCOUNT =
+      METRICS_BLOCKCACHE_PREFIX + "summary.evictioncount";
 
   /**
    * Build Micrometer Meter objects and register them with the registry
diff --git a/core/src/main/java/org/apache/accumulo/core/protobuf/ProtobufUtil.java b/core/src/main/java/org/apache/accumulo/core/protobuf/ProtobufUtil.java
deleted file mode 100644
index b7ab266..0000000
--- a/core/src/main/java/org/apache/accumulo/core/protobuf/ProtobufUtil.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.protobuf;
-
-import org.apache.accumulo.core.data.Value;
-
-import com.google.protobuf.GeneratedMessageV3;
-import com.google.protobuf.TextFormat;
-
-/**
- * Helper methods for interacting with Protocol Buffers and Accumulo
- */
-public class ProtobufUtil {
-  private static final char LEFT_BRACKET = '[', RIGHT_BRACKET = ']';
-
-  public static Value toValue(GeneratedMessageV3 msg) {
-    return new Value(msg.toByteArray());
-  }
-
-  public static String toString(GeneratedMessageV3 msg) {
-    // Too much typing
-    return LEFT_BRACKET + TextFormat.shortDebugString(msg) + RIGHT_BRACKET;
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/replication/AccumuloReplicationReplayer.java b/core/src/main/java/org/apache/accumulo/core/replication/AccumuloReplicationReplayer.java
deleted file mode 100644
index 512c732..0000000
--- a/core/src/main/java/org/apache/accumulo/core/replication/AccumuloReplicationReplayer.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.replication;
-
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.replication.thrift.RemoteReplicationException;
-import org.apache.accumulo.core.replication.thrift.WalEdits;
-
-@Deprecated(since = "2.1.0")
-public interface AccumuloReplicationReplayer {
-
-  long replicateLog(ClientContext context, String tableName, WalEdits data)
-      throws RemoteReplicationException, AccumuloException, AccumuloSecurityException;
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/replication/ReplicationConfigurationUtil.java b/core/src/main/java/org/apache/accumulo/core/replication/ReplicationConfigurationUtil.java
deleted file mode 100644
index a2ada12..0000000
--- a/core/src/main/java/org/apache/accumulo/core/replication/ReplicationConfigurationUtil.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.replication;
-
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.dataImpl.KeyExtent;
-
-/**
- * Encapsulates configuration semantics around replication
- */
-@Deprecated
-public class ReplicationConfigurationUtil {
-
-  /**
-   * Determines if the replication is enabled for the given {@link KeyExtent}
-   *
-   * @param extent The {@link KeyExtent} for the Tablet in question
-   * @param conf The {@link AccumuloConfiguration} for that Tablet (table or namespace)
-   * @return True if this extent is a candidate for replication at the given point in time.
-   */
-  public static boolean isEnabled(KeyExtent extent, AccumuloConfiguration conf) {
-    if (extent.isMeta() || extent.isRootTablet()) {
-      return false;
-    }
-
-    return conf.getBoolean(Property.TABLE_REPLICATION);
-  }
-
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/replication/ReplicationConstants.java b/core/src/main/java/org/apache/accumulo/core/replication/ReplicationConstants.java
deleted file mode 100644
index 4adde40..0000000
--- a/core/src/main/java/org/apache/accumulo/core/replication/ReplicationConstants.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.replication;
-
-@Deprecated
-public class ReplicationConstants {
-
-  // Constants for replication information in zookeeper
-  public static final String ZOO_BASE = "/replication";
-  public static final String ZOO_WORK_QUEUE = ZOO_BASE + "/workqueue";
-  public static final String ZOO_TSERVERS = ZOO_BASE + "/tservers";
-
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/replication/ReplicationSchema.java b/core/src/main/java/org/apache/accumulo/core/replication/ReplicationSchema.java
deleted file mode 100644
index e22a918..0000000
--- a/core/src/main/java/org/apache/accumulo/core/replication/ReplicationSchema.java
+++ /dev/null
@@ -1,281 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.replication;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static java.nio.charset.StandardCharsets.UTF_8;
-import static java.util.Objects.requireNonNull;
-
-import java.nio.charset.CharacterCodingException;
-
-import org.apache.accumulo.core.client.ScannerBase;
-import org.apache.accumulo.core.client.lexicoder.ULongLexicoder;
-import org.apache.accumulo.core.data.ArrayByteSequence;
-import org.apache.accumulo.core.data.ByteSequence;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.data.Value;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class ReplicationSchema {
-  private static final Logger log = LoggerFactory.getLogger(ReplicationSchema.class);
-
-  /**
-   * Portion of a file that must be replication to the given target: peer and some identifying
-   * location on that peer, e.g. remote table ID
-   *
-   * <pre>
-   * <code>
-   * hdfs://localhost:8020/accumulo/wal/tserver+port/WAL work:serialized_ReplicationTarget []
-   * -&gt; Status Protobuf
-   * </code>
-   * </pre>
-   */
-  @Deprecated
-  public static class WorkSection {
-    public static final Text NAME = new Text("work");
-    private static final ByteSequence BYTE_SEQ_NAME = new ArrayByteSequence("work");
-
-    public static void getFile(Key k, Text buff) {
-      requireNonNull(k);
-      requireNonNull(buff);
-      checkArgument(BYTE_SEQ_NAME.equals(k.getColumnFamilyData()),
-          "Given replication work key with incorrect colfam");
-      _getFile(k, buff);
-    }
-
-    public static ReplicationTarget getTarget(Key k, Text buff) {
-      checkArgument(BYTE_SEQ_NAME.equals(k.getColumnFamilyData()),
-          "Given replication work key with incorrect colfam");
-      k.getColumnQualifier(buff);
-
-      return ReplicationTarget.from(buff);
-    }
-
-    /**
-     * Limit the scanner to only pull replication work records
-     */
-    public static void limit(ScannerBase scanner) {
-      scanner.fetchColumnFamily(NAME);
-    }
-
-    public static Mutation add(Mutation m, Text serializedTarget, Value v) {
-      m.put(NAME, serializedTarget, v);
-      return m;
-    }
-  }
-
-  /**
-   * Holds replication markers tracking status for files
-   *
-   * <pre>
-   * <code>
-   * hdfs://localhost:8020/accumulo/wal/tserver+port/WAL repl:local_table_id []
-   *  -&gt; Status Protobuf
-   * </code>
-   * </pre>
-   */
-  @Deprecated
-  public static class StatusSection {
-    public static final Text NAME = new Text("repl");
-    private static final ByteSequence BYTE_SEQ_NAME = new ArrayByteSequence("repl");
-
-    /**
-     * Extract the table ID from the key (inefficiently if called repeatedly)
-     *
-     * @param k Key to extract from
-     * @return The table ID
-     */
-    public static TableId getTableId(Key k) {
-      requireNonNull(k);
-      return TableId.of(k.getColumnQualifier().toString());
-    }
-
-    /**
-     * Extract the file name from the row suffix into the given {@link Text}
-     *
-     * @param k Key to extract from
-     * @param buff Text to place file name into
-     */
-    public static void getFile(Key k, Text buff) {
-      requireNonNull(k);
-      requireNonNull(buff);
-      checkArgument(BYTE_SEQ_NAME.equals(k.getColumnFamilyData()),
-          "Given replication status key with incorrect colfam");
-
-      _getFile(k, buff);
-    }
-
-    /**
-     * Limit the scanner to only return Status records
-     */
-    public static void limit(ScannerBase scanner) {
-      scanner.fetchColumnFamily(NAME);
-    }
-
-    public static Mutation add(Mutation m, TableId tableId, Value v) {
-      m.put(NAME, new Text(tableId.canonical()), v);
-      return m;
-    }
-  }
-
-  /**
-   * Holds the order in which files needed for replication were closed. The intent is to be able to
-   * guarantee that files which were closed earlier were replicated first and we don't replay data
-   * in the wrong order on our peers
-   *
-   * <pre>
-   * <code>
-   * encodedTimeOfClosure\x00hdfs://localhost:8020/accumulo/wal/tserver+port/WAL
-   *   order:source_table_id [] -&gt; Status Protobuf
-   * </code>
-   * </pre>
-   */
-  @Deprecated
-  public static class OrderSection {
-    public static final Text NAME = new Text("order");
-    public static final Text ROW_SEPARATOR = new Text(new byte[] {0});
-    private static final ULongLexicoder longEncoder = new ULongLexicoder();
-
-    /**
-     * Extract the table ID from the given key
-     *
-     * @param k OrderSection key
-     * @param buff Text to place table ID into
-     */
-    public static void getTableId(Key k, Text buff) {
-      requireNonNull(k);
-      requireNonNull(buff);
-
-      k.getColumnQualifier(buff);
-    }
-
-    /**
-     * Limit the scanner to only return Order records
-     */
-    public static void limit(ScannerBase scanner) {
-      scanner.fetchColumnFamily(NAME);
-    }
-
-    /**
-     * Creates the Mutation for the Order section for the given file and time
-     *
-     * @param file Filename
-     * @param timeInMillis Time in millis that the file was closed
-     * @return Mutation for the Order section
-     */
-    public static Mutation createMutation(String file, long timeInMillis) {
-      requireNonNull(file);
-      checkArgument(timeInMillis >= 0, "timeInMillis must be greater than zero");
-
-      // Encode the time so it sorts properly
-      byte[] rowPrefix = longEncoder.encode(timeInMillis);
-      Text row = new Text(rowPrefix);
-
-      // Normalize the file using Path
-      Path p = new Path(file);
-      String pathString = p.toUri().toString();
-
-      log.trace("Normalized {} into {}", file, pathString);
-
-      // Append the file as a suffix to the row
-      row.append((ROW_SEPARATOR + pathString).getBytes(UTF_8), 0,
-          pathString.length() + ROW_SEPARATOR.getLength());
-
-      // Make the mutation and add the column update
-      return new Mutation(row);
-    }
-
-    /**
-     * Add a column update to the given mutation with the provided tableId and value
-     *
-     * @param m Mutation for OrderSection
-     * @param tableId Source table id
-     * @param v Serialized Status msg
-     * @return The original Mutation
-     */
-    public static Mutation add(Mutation m, TableId tableId, Value v) {
-      m.put(NAME, new Text(tableId.canonical()), v);
-      return m;
-    }
-
-    public static long getTimeClosed(Key k) {
-      return getTimeClosed(k, new Text());
-    }
-
-    public static long getTimeClosed(Key k, Text buff) {
-      k.getRow(buff);
-      int offset = 0;
-      // find the last offset
-      while (true) {
-        int nextOffset = buff.find(ROW_SEPARATOR.toString(), offset + 1);
-        if (nextOffset == -1) {
-          break;
-        }
-        offset = nextOffset;
-      }
-
-      if (offset == -1) {
-        throw new IllegalArgumentException(
-            "Row does not contain expected separator for OrderSection");
-      }
-
-      byte[] encodedLong = new byte[offset];
-      System.arraycopy(buff.getBytes(), 0, encodedLong, 0, offset);
-      return longEncoder.decode(encodedLong);
-    }
-
-    public static String getFile(Key k) {
-      Text buff = new Text();
-      return getFile(k, buff);
-    }
-
-    public static String getFile(Key k, Text buff) {
-      k.getRow(buff);
-      int offset = 0;
-      // find the last offset
-      while (true) {
-        int nextOffset = buff.find(ROW_SEPARATOR.toString(), offset + 1);
-        if (nextOffset == -1) {
-          break;
-        }
-        offset = nextOffset;
-      }
-
-      if (offset == -1) {
-        throw new IllegalArgumentException(
-            "Row does not contain expected separator for OrderSection");
-      }
-
-      try {
-        return Text.decode(buff.getBytes(), offset + 1, buff.getLength() - (offset + 1));
-      } catch (CharacterCodingException e) {
-        throw new IllegalArgumentException("Could not decode file path", e);
-      }
-    }
-  }
-
-  private static void _getFile(Key k, Text buff) {
-    k.getRow(buff);
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/replication/ReplicationTable.java b/core/src/main/java/org/apache/accumulo/core/replication/ReplicationTable.java
deleted file mode 100644
index 522393a..0000000
--- a/core/src/main/java/org/apache/accumulo/core/replication/ReplicationTable.java
+++ /dev/null
@@ -1,106 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.replication;
-
-import java.util.Collections;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.BatchScanner;
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.TableOfflineException;
-import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.clientImpl.Namespace;
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.manager.state.tables.TableState;
-import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
-import org.apache.accumulo.core.replication.ReplicationSchema.WorkSection;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.hadoop.io.Text;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-@Deprecated
-public class ReplicationTable {
-  private static final Logger log = LoggerFactory.getLogger(ReplicationTable.class);
-
-  public static final TableId ID = TableId.of("+rep");
-  public static final String NAME = Namespace.ACCUMULO.name() + ".replication";
-
-  public static final String COMBINER_NAME = "statuscombiner";
-
-  public static final String STATUS_LG_NAME = StatusSection.NAME.toString();
-  public static final Set<Text> STATUS_LG_COLFAMS = Collections.singleton(StatusSection.NAME);
-  public static final String WORK_LG_NAME = WorkSection.NAME.toString();
-  public static final Set<Text> WORK_LG_COLFAMS = Collections.singleton(WorkSection.NAME);
-  public static final Map<String,Set<Text>> LOCALITY_GROUPS =
-      Map.of(STATUS_LG_NAME, STATUS_LG_COLFAMS, WORK_LG_NAME, WORK_LG_COLFAMS);
-
-  public static Scanner getScanner(AccumuloClient client) throws ReplicationTableOfflineException {
-    try {
-      return client.createScanner(NAME, Authorizations.EMPTY);
-    } catch (TableNotFoundException e) {
-      throw new AssertionError(NAME + " should exist, but doesn't.");
-    } catch (TableOfflineException e) {
-      throw new ReplicationTableOfflineException(e);
-    }
-  }
-
-  public static BatchWriter getBatchWriter(AccumuloClient client)
-      throws ReplicationTableOfflineException {
-    try {
-      return client.createBatchWriter(NAME);
-    } catch (TableNotFoundException e) {
-      throw new AssertionError(NAME + " should exist, but doesn't.");
-    } catch (TableOfflineException e) {
-      throw new ReplicationTableOfflineException(e);
-    }
-  }
-
-  public static BatchScanner getBatchScanner(AccumuloClient client, int queryThreads)
-      throws ReplicationTableOfflineException {
-    try {
-      return client.createBatchScanner(NAME, Authorizations.EMPTY, queryThreads);
-    } catch (TableNotFoundException e) {
-      throw new AssertionError(NAME + " should exist, but doesn't.");
-    } catch (TableOfflineException e) {
-      throw new ReplicationTableOfflineException(e);
-    }
-  }
-
-  public static boolean isOnline(AccumuloClient client) {
-    return ((ClientContext) client).getTableState(ID) == TableState.ONLINE;
-  }
-
-  public static void setOnline(AccumuloClient client)
-      throws AccumuloSecurityException, AccumuloException {
-    try {
-      log.info("Bringing replication table online");
-      client.tableOperations().online(NAME, true);
-    } catch (TableNotFoundException e) {
-      throw new AssertionError(NAME + " should exist, but doesn't.");
-    }
-  }
-
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/replication/ReplicationTableOfflineException.java b/core/src/main/java/org/apache/accumulo/core/replication/ReplicationTableOfflineException.java
deleted file mode 100644
index 008423f..0000000
--- a/core/src/main/java/org/apache/accumulo/core/replication/ReplicationTableOfflineException.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.replication;
-
-import org.apache.accumulo.core.client.TableOfflineException;
-
-@Deprecated
-public class ReplicationTableOfflineException extends Exception {
-
-  private static final long serialVersionUID = 1L;
-
-  public ReplicationTableOfflineException(TableOfflineException e) {
-    super(e);
-  }
-
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/replication/ReplicationTarget.java b/core/src/main/java/org/apache/accumulo/core/replication/ReplicationTarget.java
deleted file mode 100644
index a40c867..0000000
--- a/core/src/main/java/org/apache/accumulo/core/replication/ReplicationTarget.java
+++ /dev/null
@@ -1,198 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.replication;
-
-import static java.nio.charset.StandardCharsets.UTF_8;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.accumulo.core.data.TableId;
-import org.apache.hadoop.io.DataInputBuffer;
-import org.apache.hadoop.io.DataOutputBuffer;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableUtils;
-
-/**
- * Container for where some work needs to be replicated
- */
-@Deprecated
-public class ReplicationTarget implements Writable {
-
-  private String peerName;
-  private String remoteIdentifier;
-  private TableId sourceTableId;
-
-  public ReplicationTarget() {}
-
-  public ReplicationTarget(String peerName, String remoteIdentifier, TableId sourceTableId) {
-    this.peerName = peerName;
-    this.remoteIdentifier = remoteIdentifier;
-    this.sourceTableId = sourceTableId;
-  }
-
-  public String getPeerName() {
-    return peerName;
-  }
-
-  public void setPeerName(String peerName) {
-    this.peerName = peerName;
-  }
-
-  public String getRemoteIdentifier() {
-    return remoteIdentifier;
-  }
-
-  public void setRemoteIdentifier(String remoteIdentifier) {
-    this.remoteIdentifier = remoteIdentifier;
-  }
-
-  public TableId getSourceTableId() {
-    return sourceTableId;
-  }
-
-  public void setSourceTableId(TableId sourceTableId) {
-    this.sourceTableId = sourceTableId;
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    if (peerName == null) {
-      out.writeBoolean(false);
-    } else {
-      out.writeBoolean(true);
-      WritableUtils.writeString(out, peerName);
-    }
-
-    if (remoteIdentifier == null) {
-      out.writeBoolean(false);
-    } else {
-      out.writeBoolean(true);
-      WritableUtils.writeString(out, remoteIdentifier);
-    }
-
-    if (sourceTableId == null) {
-      out.writeBoolean(false);
-    } else {
-      out.writeBoolean(true);
-      WritableUtils.writeString(out, sourceTableId.canonical());
-    }
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    if (in.readBoolean()) {
-      this.peerName = WritableUtils.readString(in);
-    }
-    if (in.readBoolean()) {
-      this.remoteIdentifier = WritableUtils.readString(in);
-    }
-    if (in.readBoolean()) {
-      this.sourceTableId = TableId.of(WritableUtils.readString(in));
-    }
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder sb = new StringBuilder(64);
-    sb.append("Remote Name: ").append(peerName).append(" Remote identifier: ")
-        .append(remoteIdentifier).append(" Source Table ID: ").append(sourceTableId);
-    return sb.toString();
-  }
-
-  @Override
-  public int hashCode() {
-    return peerName.hashCode() ^ remoteIdentifier.hashCode() ^ sourceTableId.hashCode();
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (o instanceof ReplicationTarget) {
-      ReplicationTarget other = (ReplicationTarget) o;
-
-      return peerName.equals(other.peerName) && remoteIdentifier.equals(other.remoteIdentifier)
-          && sourceTableId.equals(other.sourceTableId);
-    }
-
-    return false;
-  }
-
-  /**
-   * Deserialize a ReplicationTarget
-   *
-   * @param t Serialized copy
-   * @return the deserialized version
-   */
-  public static ReplicationTarget from(Text t) {
-    ReplicationTarget target = new ReplicationTarget();
-    DataInputBuffer buffer = new DataInputBuffer();
-    buffer.reset(t.getBytes(), t.getLength());
-
-    try {
-      target.readFields(buffer);
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-
-    return target;
-  }
-
-  /**
-   * Deserialize a ReplicationTarget
-   *
-   * @param s Serialized copy
-   * @return the deserialized version
-   */
-  public static ReplicationTarget from(String s) {
-    ReplicationTarget target = new ReplicationTarget();
-    DataInputBuffer buffer = new DataInputBuffer();
-    buffer.reset(s.getBytes(UTF_8), s.length());
-
-    try {
-      target.readFields(buffer);
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-
-    return target;
-  }
-
-  /**
-   * Convenience method to serialize a ReplicationTarget to {@link Text} using the {@link Writable}
-   * methods without caring about performance penalties due to excessive object creation
-   *
-   * @return The serialized representation of the object
-   */
-  public Text toText() {
-    DataOutputBuffer buffer = new DataOutputBuffer();
-
-    try {
-      this.write(buffer);
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-
-    Text t = new Text();
-    // Throw it in a text for the mutation
-    t.set(buffer.getData(), 0, buffer.getLength());
-    return t;
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/SaslConnectionParams.java b/core/src/main/java/org/apache/accumulo/core/rpc/SaslConnectionParams.java
index bb19d5b..0aaba98 100644
--- a/core/src/main/java/org/apache/accumulo/core/rpc/SaslConnectionParams.java
+++ b/core/src/main/java/org/apache/accumulo/core/rpc/SaslConnectionParams.java
@@ -21,9 +21,11 @@
 import static java.util.Objects.requireNonNull;
 
 import java.io.IOException;
+import java.io.UncheckedIOException;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.Objects;
 import java.util.Properties;
 
 import javax.security.auth.callback.CallbackHandler;
@@ -166,7 +168,7 @@
   protected void updatePrincipalFromUgi() {
     // Ensure we're using Kerberos auth for Hadoop UGI
     if (!UserGroupInformation.isSecurityEnabled()) {
-      throw new RuntimeException("Cannot use SASL if Hadoop security is not enabled");
+      throw new IllegalStateException("Cannot use SASL if Hadoop security is not enabled");
     }
 
     // Get the current user
@@ -174,13 +176,13 @@
     try {
       currentUser = UserGroupInformation.getCurrentUser();
     } catch (IOException e) {
-      throw new RuntimeException("Failed to get current user", e);
+      throw new UncheckedIOException("Failed to get current user", e);
     }
 
     // The full name is our principal
     this.principal = currentUser.getUserName();
     if (this.principal == null) {
-      throw new RuntimeException("Got null username from " + currentUser);
+      throw new IllegalStateException("Got null username from " + currentUser);
     }
 
   }
@@ -260,11 +262,7 @@
       if (!mechanism.equals(other.mechanism)) {
         return false;
       }
-      if (callbackHandler == null) {
-        if (other.callbackHandler != null) {
-          return false;
-        }
-      } else if (!callbackHandler.equals(other.callbackHandler)) {
+      if (!Objects.equals(callbackHandler, other.callbackHandler)) {
         return false;
       }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/TTimeoutTransport.java b/core/src/main/java/org/apache/accumulo/core/rpc/TTimeoutTransport.java
index 1e1f4a3..88f9ade 100644
--- a/core/src/main/java/org/apache/accumulo/core/rpc/TTimeoutTransport.java
+++ b/core/src/main/java/org/apache/accumulo/core/rpc/TTimeoutTransport.java
@@ -28,7 +28,6 @@
 import java.net.SocketAddress;
 import java.nio.channels.spi.SelectorProvider;
 
-import org.apache.accumulo.core.util.HostAndPort;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.thrift.transport.TIOStreamTransport;
 import org.apache.thrift.transport.TTransport;
@@ -36,6 +35,8 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.net.HostAndPort;
+
 /**
  * A utility class for setting up a {@link TTransport} with various necessary configurations for
  * ideal performance in Accumulo. These configurations include:
diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/ThriftUtil.java b/core/src/main/java/org/apache/accumulo/core/rpc/ThriftUtil.java
index 41c4650..44c5892 100644
--- a/core/src/main/java/org/apache/accumulo/core/rpc/ThriftUtil.java
+++ b/core/src/main/java/org/apache/accumulo/core/rpc/ThriftUtil.java
@@ -18,18 +18,18 @@
  */
 package org.apache.accumulo.core.rpc;
 
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
+
 import java.io.IOException;
 import java.io.UncheckedIOException;
 import java.net.InetAddress;
 import java.nio.channels.ClosedByInterruptException;
-import java.security.SecureRandom;
 import java.util.HashMap;
 import java.util.Map;
 
 import org.apache.accumulo.core.clientImpl.ClientContext;
 import org.apache.accumulo.core.rpc.SaslConnectionParams.SaslMechanism;
 import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
-import org.apache.accumulo.core.util.HostAndPort;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
 import org.apache.thrift.TException;
@@ -44,6 +44,8 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.net.HostAndPort;
+
 /**
  * Factory methods for creating Thrift client objects
  */
@@ -58,7 +60,6 @@
 
   public static final String GSSAPI = "GSSAPI", DIGEST_MD5 = "DIGEST-MD5";
 
-  private static final SecureRandom random = new SecureRandom();
   private static final int RELOGIN_MAX_BACKOFF = 5000;
 
   /**
@@ -339,7 +340,7 @@
       if (loginUser == null || !loginUser.hasKerberosCredentials()) {
         // We should have already checked that we're logged in and have credentials. A
         // precondition-like check.
-        throw new RuntimeException("Expected to find Kerberos UGI credentials, but did not");
+        throw new IllegalStateException("Expected to find Kerberos UGI credentials, but did not");
       }
       UserGroupInformation currentUser = UserGroupInformation.getCurrentUser();
       // A Proxy user is the "effective user" (in name only), riding on top of the "real user"'s Krb
@@ -358,7 +359,7 @@
 
         // Avoid the replay attack protection, sleep 1 to 5000ms
         try {
-          Thread.sleep(random.nextInt(RELOGIN_MAX_BACKOFF) + 1);
+          Thread.sleep(RANDOM.get().nextInt(RELOGIN_MAX_BACKOFF) + 1);
         } catch (InterruptedException e) {
           Thread.currentThread().interrupt();
           return;
@@ -371,7 +372,7 @@
       // The inability to check is worrisome and deserves a RuntimeException instead of a propagated
       // IO-like Exception.
       log.warn("Failed to check (and/or perform) Kerberos client re-login", e);
-      throw new RuntimeException(e);
+      throw new UncheckedIOException(e);
     }
   }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/UGIAssumingTransport.java b/core/src/main/java/org/apache/accumulo/core/rpc/UGIAssumingTransport.java
index 913cd19..133999c 100644
--- a/core/src/main/java/org/apache/accumulo/core/rpc/UGIAssumingTransport.java
+++ b/core/src/main/java/org/apache/accumulo/core/rpc/UGIAssumingTransport.java
@@ -19,6 +19,7 @@
 package org.apache.accumulo.core.rpc;
 
 import java.io.IOException;
+import java.io.UncheckedIOException;
 import java.security.PrivilegedExceptionAction;
 import java.util.concurrent.atomic.AtomicReference;
 
@@ -56,8 +57,10 @@
         }
         return null;
       });
-    } catch (IOException | InterruptedException e) {
-      throw new RuntimeException(e);
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    } catch (InterruptedException e) {
+      throw new IllegalStateException(e);
     }
 
     // Make sure the transport exception gets (re)thrown if it happened
diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/UGIAssumingTransportFactory.java b/core/src/main/java/org/apache/accumulo/core/rpc/UGIAssumingTransportFactory.java
index e9eb384..707a41f 100644
--- a/core/src/main/java/org/apache/accumulo/core/rpc/UGIAssumingTransportFactory.java
+++ b/core/src/main/java/org/apache/accumulo/core/rpc/UGIAssumingTransportFactory.java
@@ -52,7 +52,7 @@
       try {
         return wrapped.getTransport(trans);
       } catch (TTransportException e) {
-        throw new RuntimeException(e);
+        throw new IllegalStateException(e);
       }
     });
   }
diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/clients/ClientServiceThriftClient.java b/core/src/main/java/org/apache/accumulo/core/rpc/clients/ClientServiceThriftClient.java
index 5a6a912..c1dc995 100644
--- a/core/src/main/java/org/apache/accumulo/core/rpc/clients/ClientServiceThriftClient.java
+++ b/core/src/main/java/org/apache/accumulo/core/rpc/clients/ClientServiceThriftClient.java
@@ -24,6 +24,7 @@
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.clientImpl.ClientContext;
 import org.apache.accumulo.core.clientImpl.thrift.ClientService.Client;
+import org.apache.accumulo.core.lock.ServiceLockData.ThriftService;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.thrift.transport.TTransportException;
 import org.slf4j.Logger;
@@ -40,10 +41,10 @@
   }
 
   @Override
-  public Pair<String,Client> getTabletServerConnection(ClientContext context,
+  public Pair<String,Client> getThriftServerConnection(ClientContext context,
       boolean preferCachedConnections) throws TTransportException {
-    return getTabletServerConnection(LOG, this, context, preferCachedConnections,
-        warnedAboutTServersBeingDown);
+    return getThriftServerConnection(LOG, this, context, preferCachedConnections,
+        warnedAboutTServersBeingDown, ThriftService.CLIENT);
   }
 
   @Override
diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/clients/ManagerClient.java b/core/src/main/java/org/apache/accumulo/core/rpc/clients/ManagerClient.java
index 2561d60..d0076e6 100644
--- a/core/src/main/java/org/apache/accumulo/core/rpc/clients/ManagerClient.java
+++ b/core/src/main/java/org/apache/accumulo/core/rpc/clients/ManagerClient.java
@@ -25,11 +25,12 @@
 
 import org.apache.accumulo.core.clientImpl.ClientContext;
 import org.apache.accumulo.core.rpc.ThriftUtil;
-import org.apache.accumulo.core.util.HostAndPort;
 import org.apache.thrift.TServiceClient;
 import org.apache.thrift.transport.TTransportException;
 import org.slf4j.Logger;
 
+import com.google.common.net.HostAndPort;
+
 public interface ManagerClient<C extends TServiceClient> {
 
   default C getManagerConnection(Logger log, ThriftClientTypes<C> type, ClientContext context) {
@@ -54,7 +55,7 @@
       Throwable cause = tte.getCause();
       if (cause != null && cause instanceof UnknownHostException) {
         // do not expect to recover from this
-        throw new RuntimeException(tte);
+        throw new IllegalStateException(tte);
       }
       log.debug("Failed to connect to manager=" + manager + ", will retry... ", tte);
       return null;
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 21a3b1c..eb353fe 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
@@ -18,8 +18,8 @@
  */
 package org.apache.accumulo.core.rpc.clients;
 
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
 
 import java.util.ConcurrentModificationException;
 
diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/clients/ReplicationCoordinatorThriftClient.java b/core/src/main/java/org/apache/accumulo/core/rpc/clients/ReplicationCoordinatorThriftClient.java
deleted file mode 100644
index da6457d..0000000
--- a/core/src/main/java/org/apache/accumulo/core/rpc/clients/ReplicationCoordinatorThriftClient.java
+++ /dev/null
@@ -1,150 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.rpc.clients;
-
-import static java.nio.charset.StandardCharsets.UTF_8;
-import static java.util.Objects.requireNonNull;
-
-import java.util.List;
-
-import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
-import org.apache.accumulo.core.fate.zookeeper.ZooReader;
-import org.apache.accumulo.core.replication.thrift.ReplicationCoordinator.Client;
-import org.apache.accumulo.core.rpc.ThriftUtil;
-import org.apache.accumulo.core.util.HostAndPort;
-import org.apache.thrift.transport.TTransportException;
-import org.apache.zookeeper.KeeperException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class ReplicationCoordinatorThriftClient extends ThriftClientTypes<Client> {
-
-  private static final Logger LOG =
-      LoggerFactory.getLogger(ReplicationCoordinatorThriftClient.class);
-
-  ReplicationCoordinatorThriftClient(String serviceName) {
-    super(serviceName, new Client.Factory());
-  }
-
-  @Override
-  public Client getConnection(ClientContext context) {
-
-    List<String> locations = context.getManagerLocations();
-
-    if (locations.isEmpty()) {
-      LOG.debug("No managers for replication to instance {}", context.getInstanceName());
-      return null;
-    }
-
-    // This is the manager thrift service, we just want the hostname, not the port
-    String managerThriftService = locations.get(0);
-    if (managerThriftService.endsWith(":0")) {
-      LOG.warn("Manager found for {} did not have real location {}", context.getInstanceName(),
-          managerThriftService);
-      return null;
-    }
-
-    String zkPath = context.getZooKeeperRoot() + Constants.ZMANAGER_REPLICATION_COORDINATOR_ADDR;
-    String replCoordinatorAddr;
-
-    LOG.debug("Using ZooKeeper quorum at {} with path {} to find peer Manager information",
-        context.getZooKeepers(), zkPath);
-
-    // Get the coordinator port for the manager we're trying to connect to
-    try {
-      ZooReader reader = context.getZooReader();
-      replCoordinatorAddr = new String(reader.getData(zkPath), UTF_8);
-    } catch (KeeperException | InterruptedException e) {
-      LOG.error("Could not fetch remote coordinator port", e);
-      return null;
-    }
-
-    // Throw the hostname and port through HostAndPort to get some normalization
-    HostAndPort coordinatorAddr = HostAndPort.fromString(replCoordinatorAddr);
-
-    LOG.debug("Connecting to manager at {}", coordinatorAddr);
-
-    try {
-      // Manager requests can take a long time: don't ever time out
-      return ThriftUtil.getClientNoTimeout(ThriftClientTypes.REPLICATION_COORDINATOR,
-          coordinatorAddr, context);
-    } catch (TTransportException tte) {
-      LOG.debug("Failed to connect to manager coordinator service ({})", coordinatorAddr, tte);
-      return null;
-    }
-  }
-
-  @Override
-  public Client getConnectionWithRetry(ClientContext context) {
-    requireNonNull(context);
-
-    for (int attempts = 1; attempts <= 10; attempts++) {
-
-      Client result = getConnection(context);
-      if (result != null) {
-        return result;
-      }
-      LOG.debug("Could not get ReplicationCoordinator connection to {}, will retry",
-          context.getInstanceName());
-      try {
-        Thread.sleep(attempts * 250L);
-      } catch (InterruptedException e) {
-        throw new RuntimeException(e);
-      }
-    }
-
-    throw new RuntimeException(
-        "Timed out trying to communicate with manager from " + context.getInstanceName());
-  }
-
-  @Override
-  public <R> R execute(ClientContext context, Exec<R,Client> exec)
-      throws AccumuloException, AccumuloSecurityException {
-    Client client = null;
-    for (int i = 0; i < 10; i++) {
-      try {
-        client = getConnectionWithRetry(context);
-        return exec.execute(client);
-      } catch (TTransportException tte) {
-        LOG.debug("ReplicationClient coordinator request failed, retrying ... ", tte);
-        try {
-          Thread.sleep(100);
-        } catch (InterruptedException e) {
-          throw new AccumuloException(e);
-        }
-      } catch (ThriftSecurityException e) {
-        throw new AccumuloSecurityException(e.user, e.code, e);
-      } catch (Exception e) {
-        throw new AccumuloException(e);
-      } finally {
-        if (client != null) {
-          ThriftUtil.close(client, context);
-        }
-      }
-    }
-
-    throw new AccumuloException(
-        "Could not connect to ReplicationCoordinator at " + context.getInstanceName());
-  }
-
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/clients/TServerClient.java b/core/src/main/java/org/apache/accumulo/core/rpc/clients/TServerClient.java
index 5f01766..4027f4b 100644
--- a/core/src/main/java/org/apache/accumulo/core/rpc/clients/TServerClient.java
+++ b/core/src/main/java/org/apache/accumulo/core/rpc/clients/TServerClient.java
@@ -19,13 +19,14 @@
 package org.apache.accumulo.core.rpc.clients;
 
 import static com.google.common.base.Preconditions.checkArgument;
-import static java.nio.charset.StandardCharsets.UTF_8;
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
+import java.util.Optional;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.accumulo.core.Constants;
@@ -34,15 +35,14 @@
 import org.apache.accumulo.core.clientImpl.AccumuloServerException;
 import org.apache.accumulo.core.clientImpl.ClientContext;
 import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock;
 import org.apache.accumulo.core.fate.zookeeper.ZooCache;
+import org.apache.accumulo.core.lock.ServiceLock;
+import org.apache.accumulo.core.lock.ServiceLockData;
+import org.apache.accumulo.core.lock.ServiceLockData.ThriftService;
 import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.rpc.clients.ThriftClientTypes.Exec;
 import org.apache.accumulo.core.rpc.clients.ThriftClientTypes.ExecVoid;
-import org.apache.accumulo.core.util.HostAndPort;
 import org.apache.accumulo.core.util.Pair;
-import org.apache.accumulo.core.util.ServerServices;
-import org.apache.accumulo.core.util.ServerServices.Service;
 import org.apache.thrift.TApplicationException;
 import org.apache.thrift.TException;
 import org.apache.thrift.TServiceClient;
@@ -50,14 +50,16 @@
 import org.apache.thrift.transport.TTransportException;
 import org.slf4j.Logger;
 
+import com.google.common.net.HostAndPort;
+
 public interface TServerClient<C extends TServiceClient> {
 
-  Pair<String,C> getTabletServerConnection(ClientContext context, boolean preferCachedConnections)
+  Pair<String,C> getThriftServerConnection(ClientContext context, boolean preferCachedConnections)
       throws TTransportException;
 
-  default Pair<String,C> getTabletServerConnection(Logger LOG, ThriftClientTypes<C> type,
-      ClientContext context, boolean preferCachedConnections, AtomicBoolean warned)
-      throws TTransportException {
+  default Pair<String,C> getThriftServerConnection(Logger LOG, ThriftClientTypes<C> type,
+      ClientContext context, boolean preferCachedConnections, AtomicBoolean warned,
+      ThriftService service) throws TTransportException {
     checkArgument(context != null, "context is null");
 
     if (preferCachedConnections) {
@@ -71,29 +73,42 @@
     }
 
     final long rpcTimeout = context.getClientTimeoutInMillis();
+    final String tserverZooPath = context.getZooKeeperRoot() + Constants.ZTSERVERS;
+    final String sserverZooPath = context.getZooKeeperRoot() + Constants.ZSSERVERS;
+    final String compactorZooPath = context.getZooKeeperRoot() + Constants.ZCOMPACTORS;
     final ZooCache zc = context.getZooCache();
-    final List<String> tservers = new ArrayList<>();
 
-    tservers.addAll(zc.getChildren(context.getZooKeeperRoot() + Constants.ZTSERVERS));
+    final List<String> serverPaths = new ArrayList<>();
+    zc.getChildren(tserverZooPath).forEach(tserverAddress -> {
+      serverPaths.add(tserverZooPath + "/" + tserverAddress);
+    });
+    if (type == ThriftClientTypes.CLIENT) {
+      zc.getChildren(sserverZooPath).forEach(sserverAddress -> {
+        serverPaths.add(sserverZooPath + "/" + sserverAddress);
+      });
+      zc.getChildren(compactorZooPath).forEach(compactorGroup -> {
+        zc.getChildren(compactorZooPath + "/" + compactorGroup).forEach(compactorAddress -> {
+          serverPaths.add(compactorZooPath + "/" + compactorGroup + "/" + compactorAddress);
+        });
+      });
+    }
 
-    if (tservers.isEmpty()) {
+    if (serverPaths.isEmpty()) {
       if (warned.compareAndSet(false, true)) {
-        LOG.warn("There are no tablet servers: check that zookeeper and accumulo are running.");
+        LOG.warn(
+            "There are no servers serving the {} api: check that zookeeper and accumulo are running.",
+            type);
       }
       throw new TTransportException("There are no servers for type: " + type);
     }
+    Collections.shuffle(serverPaths, RANDOM.get());
 
-    // Try to connect to an online tserver
-    Collections.shuffle(tservers);
-    for (String tserver : tservers) {
-      var zLocPath =
-          ServiceLock.path(context.getZooKeeperRoot() + Constants.ZTSERVERS + "/" + tserver);
-      byte[] data = zc.getLockData(zLocPath);
-      if (data != null) {
-        String strData = new String(data, UTF_8);
-        if (!strData.equals("manager")) {
-          final HostAndPort tserverClientAddress =
-              new ServerServices(strData).getAddress(Service.TSERV_CLIENT);
+    for (String serverPath : serverPaths) {
+      var zLocPath = ServiceLock.path(serverPath);
+      Optional<ServiceLockData> data = zc.getLockData(zLocPath);
+      if (data != null && data.isPresent()) {
+        HostAndPort tserverClientAddress = data.orElseThrow().getAddress(service);
+        if (tserverClientAddress != null) {
           try {
             TTransport transport = context.getTransportPool().getTransport(type,
                 tserverClientAddress, rpcTimeout, context, preferCachedConnections);
@@ -107,9 +122,10 @@
         }
       }
     }
+
     if (warned.compareAndSet(false, true)) {
       LOG.warn("Failed to find an available server in the list of servers: {} for API type: {}",
-          tservers, type);
+          serverPaths, type);
     }
     throw new TTransportException("Failed to connect to any server for API type " + type);
   }
@@ -120,7 +136,7 @@
       String server = null;
       C client = null;
       try {
-        Pair<String,C> pair = getTabletServerConnection(context, true);
+        Pair<String,C> pair = getThriftServerConnection(context, true);
         server = pair.getFirst();
         client = pair.getSecond();
         return exec.execute(client);
@@ -147,7 +163,7 @@
       String server = null;
       C client = null;
       try {
-        Pair<String,C> pair = getTabletServerConnection(context, true);
+        Pair<String,C> pair = getThriftServerConnection(context, true);
         server = pair.getFirst();
         client = pair.getSecond();
         exec.execute(client);
diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/clients/ReplicationServicerThriftClient.java b/core/src/main/java/org/apache/accumulo/core/rpc/clients/TabletIngestClientServiceThriftClient.java
similarity index 67%
copy from core/src/main/java/org/apache/accumulo/core/rpc/clients/ReplicationServicerThriftClient.java
copy to core/src/main/java/org/apache/accumulo/core/rpc/clients/TabletIngestClientServiceThriftClient.java
index d347979..a34ecf7 100644
--- a/core/src/main/java/org/apache/accumulo/core/rpc/clients/ReplicationServicerThriftClient.java
+++ b/core/src/main/java/org/apache/accumulo/core/rpc/clients/TabletIngestClientServiceThriftClient.java
@@ -18,11 +18,15 @@
  */
 package org.apache.accumulo.core.rpc.clients;
 
-import org.apache.accumulo.core.replication.thrift.ReplicationServicer.Client;
+import org.apache.accumulo.core.tabletingest.thrift.TabletIngestClientService.Client;
 
-public class ReplicationServicerThriftClient extends ThriftClientTypes<Client> {
+/**
+ * Client side object that can be used to interact with services that support ingest operations
+ * against tablets. See TabletIngestClientService$Iface for a list of supported operations.
+ */
+public class TabletIngestClientServiceThriftClient extends ThriftClientTypes<Client> {
 
-  ReplicationServicerThriftClient(String serviceName) {
+  public TabletIngestClientServiceThriftClient(String serviceName) {
     super(serviceName, new Client.Factory());
   }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/clients/ReplicationServicerThriftClient.java b/core/src/main/java/org/apache/accumulo/core/rpc/clients/TabletManagementClientServiceThriftClient.java
similarity index 66%
copy from core/src/main/java/org/apache/accumulo/core/rpc/clients/ReplicationServicerThriftClient.java
copy to core/src/main/java/org/apache/accumulo/core/rpc/clients/TabletManagementClientServiceThriftClient.java
index d347979..d19a2a4 100644
--- a/core/src/main/java/org/apache/accumulo/core/rpc/clients/ReplicationServicerThriftClient.java
+++ b/core/src/main/java/org/apache/accumulo/core/rpc/clients/TabletManagementClientServiceThriftClient.java
@@ -18,12 +18,15 @@
  */
 package org.apache.accumulo.core.rpc.clients;
 
-import org.apache.accumulo.core.replication.thrift.ReplicationServicer.Client;
+import org.apache.accumulo.core.tablet.thrift.TabletManagementClientService.Client;
 
-public class ReplicationServicerThriftClient extends ThriftClientTypes<Client> {
+/**
+ * Client side object that can be used to interact with services that support management operations
+ * against tablets. See TabletManagementClientService$Iface for a list of supported operations.
+ */
+public class TabletManagementClientServiceThriftClient extends ThriftClientTypes<Client> {
 
-  ReplicationServicerThriftClient(String serviceName) {
+  public TabletManagementClientServiceThriftClient(String serviceName) {
     super(serviceName, new Client.Factory());
   }
-
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/clients/TabletScanClientServiceThriftClient.java b/core/src/main/java/org/apache/accumulo/core/rpc/clients/TabletScanClientServiceThriftClient.java
index 55f7d53..15e9ce4 100644
--- a/core/src/main/java/org/apache/accumulo/core/rpc/clients/TabletScanClientServiceThriftClient.java
+++ b/core/src/main/java/org/apache/accumulo/core/rpc/clients/TabletScanClientServiceThriftClient.java
@@ -18,8 +18,12 @@
  */
 package org.apache.accumulo.core.rpc.clients;
 
-import org.apache.accumulo.core.tabletserver.thrift.TabletScanClientService.Client;
+import org.apache.accumulo.core.tabletscan.thrift.TabletScanClientService.Client;
 
+/**
+ * Client side object that can be used to interact with services that support scan operations
+ * against tablets. See TabletScanClientService$Iface for a list of supported operations.
+ */
 public class TabletScanClientServiceThriftClient extends ThriftClientTypes<Client> {
 
   TabletScanClientServiceThriftClient(String serviceName) {
diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/clients/TabletServerThriftClient.java b/core/src/main/java/org/apache/accumulo/core/rpc/clients/TabletServerThriftClient.java
index 647bdec..3f2f71b 100644
--- a/core/src/main/java/org/apache/accumulo/core/rpc/clients/TabletServerThriftClient.java
+++ b/core/src/main/java/org/apache/accumulo/core/rpc/clients/TabletServerThriftClient.java
@@ -23,12 +23,17 @@
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Client;
+import org.apache.accumulo.core.lock.ServiceLockData.ThriftService;
+import org.apache.accumulo.core.tabletserver.thrift.TabletServerClientService.Client;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.thrift.transport.TTransportException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+/**
+ * Client side object that can be used to interact with services that support operations against
+ * TabletServers. See TabletServerThriftClient$Iface for a list of supported operations.
+ */
 public class TabletServerThriftClient extends ThriftClientTypes<Client>
     implements TServerClient<Client> {
 
@@ -40,10 +45,10 @@
   }
 
   @Override
-  public Pair<String,Client> getTabletServerConnection(ClientContext context,
+  public Pair<String,Client> getThriftServerConnection(ClientContext context,
       boolean preferCachedConnections) throws TTransportException {
-    return getTabletServerConnection(LOG, this, context, preferCachedConnections,
-        warnedAboutTServersBeingDown);
+    return getThriftServerConnection(LOG, this, context, preferCachedConnections,
+        warnedAboutTServersBeingDown, ThriftService.TSERV);
   }
 
   @Override
diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/clients/ThriftClientTypes.java b/core/src/main/java/org/apache/accumulo/core/rpc/clients/ThriftClientTypes.java
index 9559550..d7f34c4 100644
--- a/core/src/main/java/org/apache/accumulo/core/rpc/clients/ThriftClientTypes.java
+++ b/core/src/main/java/org/apache/accumulo/core/rpc/clients/ThriftClientTypes.java
@@ -18,8 +18,8 @@
  */
 package org.apache.accumulo.core.rpc.clients;
 
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
 
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -46,18 +46,18 @@
 
   public static final ManagerThriftClient MANAGER = new ManagerThriftClient("mgr");
 
-  public static final ReplicationCoordinatorThriftClient REPLICATION_COORDINATOR =
-      new ReplicationCoordinatorThriftClient("replCoord");
-
-  public static final ReplicationServicerThriftClient REPLICATION_SERVICER =
-      new ReplicationServicerThriftClient("replServ");
-
   public static final TabletServerThriftClient TABLET_SERVER =
-      new TabletServerThriftClient("tablet");
+      new TabletServerThriftClient("tserver");
 
   public static final TabletScanClientServiceThriftClient TABLET_SCAN =
       new TabletScanClientServiceThriftClient("scan");
 
+  public static final TabletIngestClientServiceThriftClient TABLET_INGEST =
+      new TabletIngestClientServiceThriftClient("ingest");
+
+  public static final TabletManagementClientServiceThriftClient TABLET_MGMT =
+      new TabletManagementClientServiceThriftClient("tablet");
+
   /**
    * execute method with supplied client returning object of type R
    *
diff --git a/core/src/main/java/org/apache/accumulo/core/sample/impl/SamplerConfigurationImpl.java b/core/src/main/java/org/apache/accumulo/core/sample/impl/SamplerConfigurationImpl.java
index d77c5a4..a97bf5e 100644
--- a/core/src/main/java/org/apache/accumulo/core/sample/impl/SamplerConfigurationImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/sample/impl/SamplerConfigurationImpl.java
@@ -32,7 +32,7 @@
 import org.apache.accumulo.core.client.sample.SamplerConfiguration;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.tabletserver.thrift.TSamplerConfiguration;
+import org.apache.accumulo.core.tabletscan.thrift.TSamplerConfiguration;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.hadoop.io.Writable;
 
diff --git a/core/src/main/java/org/apache/accumulo/core/security/Authorizations.java b/core/src/main/java/org/apache/accumulo/core/security/Authorizations.java
index 1cfeebb..380a183 100644
--- a/core/src/main/java/org/apache/accumulo/core/security/Authorizations.java
+++ b/core/src/main/java/org/apache/accumulo/core/security/Authorizations.java
@@ -385,4 +385,21 @@
 
     return sb.toString();
   }
+
+  /**
+   * Converts to an Accumulo Access Authorizations object.
+   *
+   * @since 3.1.0
+   */
+  public org.apache.accumulo.access.Authorizations toAccessAuthorizations() {
+    if (auths.isEmpty()) {
+      return org.apache.accumulo.access.Authorizations.of();
+    } else {
+      Set<String> auths = new HashSet<>(authsList.size());
+      for (var auth : authsList) {
+        auths.add(new String(auth, UTF_8));
+      }
+      return org.apache.accumulo.access.Authorizations.of(auths);
+    }
+  }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/security/ColumnVisibility.java b/core/src/main/java/org/apache/accumulo/core/security/ColumnVisibility.java
index 94772fe..bc49bac 100644
--- a/core/src/main/java/org/apache/accumulo/core/security/ColumnVisibility.java
+++ b/core/src/main/java/org/apache/accumulo/core/security/ColumnVisibility.java
@@ -27,7 +27,10 @@
 import java.util.Comparator;
 import java.util.List;
 import java.util.TreeSet;
+import java.util.function.Supplier;
 
+import org.apache.accumulo.access.AccessExpression;
+import org.apache.accumulo.access.InvalidAccessExpressionException;
 import org.apache.accumulo.core.data.ArrayByteSequence;
 import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.util.BadArgumentException;
@@ -35,6 +38,8 @@
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.WritableComparator;
 
+import com.google.common.base.Suppliers;
+
 /**
  * Validate the column visibility is a valid expression and set the visibility for a Mutation. See
  * {@link ColumnVisibility#ColumnVisibility(byte[])} for the definition of an expression.
@@ -76,8 +81,10 @@
  */
 public class ColumnVisibility {
 
-  Node node = null;
-  private byte[] expression;
+  // This functionality is deprecated so its setup as a supplier so it is only computed if the
+  // deprecated functionality is called.
+  private final Supplier<Node> nodeSupplier;
+  private final byte[] expression;
 
   /**
    * Accessor for the underlying byte string.
@@ -91,7 +98,8 @@
   /**
    * The node types in a parse tree for a visibility expression.
    */
-  public static enum NodeType {
+  @Deprecated(since = "3.1.0")
+  public enum NodeType {
     EMPTY, TERM, OR, AND,
   }
 
@@ -103,6 +111,7 @@
   /**
    * A node in the parse tree for a visibility expression.
    */
+  @Deprecated(since = "3.1.0")
   public static class Node {
     /**
      * An empty list of nodes.
@@ -151,7 +160,7 @@
 
     public ByteSequence getTerm(byte[] expression) {
       if (type != NodeType.TERM) {
-        throw new RuntimeException();
+        throw new IllegalStateException();
       }
 
       if (expression[start] == '"') {
@@ -169,6 +178,7 @@
    * A node comparator. Nodes sort according to node type, terms sort lexicographically. AND and OR
    * nodes sort by number of children, or if the same by corresponding children.
    */
+  @Deprecated(since = "3.1.0")
   public static class NodeComparator implements Comparator<Node>, Serializable {
 
     private static final long serialVersionUID = 1L;
@@ -216,6 +226,7 @@
    * Convenience method that delegates to normalize with a new NodeComparator constructed using the
    * supplied expression.
    */
+  @Deprecated(since = "3.1.0")
   public static Node normalize(Node root, byte[] expression) {
     return normalize(root, expression, new NodeComparator(expression));
   }
@@ -228,6 +239,7 @@
    *  3) dedupes labels (`a&b&a` becomes `a&b`)
    */
   // @formatter:on
+  @Deprecated(since = "3.1.0")
   public static Node normalize(Node root, byte[] expression, NodeComparator comparator) {
     if (root.type != NodeType.TERM) {
       TreeSet<Node> rolledUp = new TreeSet<>(comparator);
@@ -256,6 +268,7 @@
    * Walks an expression's AST and appends a string representation to a supplied StringBuilder. This
    * method adds parens where necessary.
    */
+  @Deprecated(since = "3.1.0")
   public static void stringify(Node root, byte[] expression, StringBuilder out) {
     if (root.type == NodeType.TERM) {
       out.append(new String(expression, root.start, root.end - root.start, UTF_8));
@@ -282,13 +295,12 @@
    *
    * @return normalized expression in byte[] form
    */
+  @Deprecated(since = "3.1.0")
   public byte[] flatten() {
-    Node normRoot = normalize(node, expression);
-    StringBuilder builder = new StringBuilder(expression.length);
-    stringify(normRoot, expression, builder);
-    return builder.toString().getBytes(UTF_8);
+    return AccessExpression.of(expression, true).getExpression().getBytes(UTF_8);
   }
 
+  @Deprecated
   private static class ColumnVisibilityParser {
     private int index = 0;
     private int parens = 0;
@@ -455,16 +467,17 @@
     }
   }
 
-  private void validate(byte[] expression) {
+  private Node createNodeTree(byte[] expression) {
     if (expression != null && expression.length > 0) {
       ColumnVisibilityParser p = new ColumnVisibilityParser();
-      node = p.parse(expression);
+      return p.parse(expression);
     } else {
-      node = EMPTY_NODE;
+      return EMPTY_NODE;
     }
-    this.expression = expression;
   }
 
+  private static final byte[] EMPTY_BYTES = new byte[0];
+
   /**
    * Creates an empty visibility. Normally, elements with empty visibility can be seen by everyone.
    * Though, one could change this behavior with filters.
@@ -472,7 +485,8 @@
    * @see #ColumnVisibility(String)
    */
   public ColumnVisibility() {
-    this(new byte[] {});
+    expression = EMPTY_BYTES;
+    nodeSupplier = Suppliers.memoize(() -> createNodeTree(expression));
   }
 
   /**
@@ -496,13 +510,34 @@
   }
 
   /**
-   * Creates a column visibility for a Mutation from a string already encoded in UTF-8 bytes.
+   * Creates a column visibility for a Mutation from bytes already encoded in UTF-8.
    *
    * @param expression visibility expression, encoded as UTF-8 bytes
    * @see #ColumnVisibility(String)
    */
   public ColumnVisibility(byte[] expression) {
-    validate(expression);
+    this.expression = expression;
+    try {
+      AccessExpression.validate(this.expression);
+    } catch (InvalidAccessExpressionException e) {
+      // This is thrown for compatability with the exception this class used to throw when it parsed
+      // exceptions itself.
+      throw new BadArgumentException(e);
+    }
+    nodeSupplier = Suppliers.memoize(() -> createNodeTree(this.expression));
+  }
+
+  /**
+   * Creates a column visibility for a Mutation from an AccessExpression.
+   *
+   * @param expression visibility expression, encoded as UTF-8 bytes
+   * @see #ColumnVisibility(String)
+   * @since 3.1.0
+   */
+  public ColumnVisibility(AccessExpression expression) {
+    // AccessExpression is a validated immutable object, so no need to re validate
+    this.expression = expression.getExpression().getBytes(UTF_8);
+    nodeSupplier = Suppliers.memoize(() -> createNodeTree(this.expression));
   }
 
   @Override
@@ -542,8 +577,9 @@
    *
    * @return parse tree node
    */
+  @Deprecated(since = "3.1.0")
   public Node getParseTree() {
-    return node;
+    return nodeSupplier.get();
   }
 
   /**
@@ -564,9 +600,11 @@
    *
    * @param term term to quote
    * @return quoted term (unquoted if unnecessary)
+   * @deprecated use {@link AccessExpression#quote(String)}
    */
+  @Deprecated(since = "3.1.0")
   public static String quote(String term) {
-    return new String(quote(term.getBytes(UTF_8)), UTF_8);
+    return AccessExpression.quote(term);
   }
 
   /**
@@ -576,21 +614,10 @@
    * @param term term to quote, encoded as UTF-8 bytes
    * @return quoted term (unquoted if unnecessary), encoded as UTF-8 bytes
    * @see #quote(String)
+   * @deprecated use {@link AccessExpression#quote(byte[])}
    */
+  @Deprecated(since = "3.1.0")
   public static byte[] quote(byte[] term) {
-    boolean needsQuote = false;
-
-    for (byte b : term) {
-      if (!Authorizations.isValidAuthChar(b)) {
-        needsQuote = true;
-        break;
-      }
-    }
-
-    if (!needsQuote) {
-      return term;
-    }
-
-    return VisibilityEvaluator.escape(term, true);
+    return AccessExpression.quote(term);
   }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/security/VisibilityEvaluator.java b/core/src/main/java/org/apache/accumulo/core/security/VisibilityEvaluator.java
index dd07d64..3c97c26 100644
--- a/core/src/main/java/org/apache/accumulo/core/security/VisibilityEvaluator.java
+++ b/core/src/main/java/org/apache/accumulo/core/security/VisibilityEvaluator.java
@@ -18,92 +18,18 @@
  */
 package org.apache.accumulo.core.security;
 
-import java.util.ArrayList;
-
+import org.apache.accumulo.access.AccessEvaluator;
+import org.apache.accumulo.access.InvalidAccessExpressionException;
 import org.apache.accumulo.core.data.ArrayByteSequence;
-import org.apache.accumulo.core.data.ByteSequence;
-import org.apache.accumulo.core.security.ColumnVisibility.Node;
 
 /**
  * A class which evaluates visibility expressions against a set of authorizations.
+ *
+ * @deprecated since 3.1.0 Use Accumulo Access library instead
  */
+@Deprecated(since = "3.1.0")
 public class VisibilityEvaluator {
-  private AuthorizationContainer auths;
-
-  /**
-   * Authorizations in column visibility expression are in escaped form. Column visibility parsing
-   * does not unescape. This class wraps an AuthorizationContainer and unescapes auths before
-   * checking the wrapped container.
-   */
-  private static class UnescapingAuthorizationContainer implements AuthorizationContainer {
-
-    private AuthorizationContainer wrapped;
-
-    UnescapingAuthorizationContainer(AuthorizationContainer wrapee) {
-      this.wrapped = wrapee;
-    }
-
-    @Override
-    public boolean contains(ByteSequence auth) {
-      return wrapped.contains(unescape(auth));
-    }
-  }
-
-  static ByteSequence unescape(ByteSequence auth) {
-    int escapeCharCount = 0;
-    for (int i = 0; i < auth.length(); i++) {
-      byte b = auth.byteAt(i);
-      if (b == '"' || b == '\\') {
-        escapeCharCount++;
-      }
-    }
-
-    if (escapeCharCount > 0) {
-      if (escapeCharCount % 2 == 1) {
-        throw new IllegalArgumentException("Illegal escape sequence in auth : " + auth);
-      }
-
-      byte[] unescapedCopy = new byte[auth.length() - escapeCharCount / 2];
-      int pos = 0;
-      for (int i = 0; i < auth.length(); i++) {
-        byte b = auth.byteAt(i);
-        if (b == '\\') {
-          i++;
-          b = auth.byteAt(i);
-          if (b != '"' && b != '\\') {
-            throw new IllegalArgumentException("Illegal escape sequence in auth : " + auth);
-          }
-        } else if (b == '"') {
-          // should only see quote after a slash
-          throw new IllegalArgumentException("Illegal escape sequence in auth : " + auth);
-        }
-
-        unescapedCopy[pos++] = b;
-      }
-
-      return new ArrayByteSequence(unescapedCopy);
-    } else {
-      return auth;
-    }
-  }
-
-  /**
-   * Creates a new {@link Authorizations} object with escaped forms of the authorizations in the
-   * given object.
-   *
-   * @param auths original authorizations
-   * @return authorizations object with escaped authorization strings
-   * @see #escape(byte[], boolean)
-   */
-  static Authorizations escape(Authorizations auths) {
-    ArrayList<byte[]> retAuths = new ArrayList<>(auths.getAuthorizations().size());
-
-    for (byte[] auth : auths.getAuthorizations()) {
-      retAuths.add(escape(auth, false));
-    }
-
-    return new Authorizations(retAuths);
-  }
+  private final AccessEvaluator accessEvaluator;
 
   /**
    * Properly escapes an authorization string. The string can be quoted if desired.
@@ -147,7 +73,9 @@
    * @since 1.7.0
    */
   public VisibilityEvaluator(AuthorizationContainer authsContainer) {
-    this.auths = new UnescapingAuthorizationContainer(authsContainer);
+    // TODO need to look into efficiency and correctness of this
+    this.accessEvaluator =
+        AccessEvaluator.of(auth -> authsContainer.contains(new ArrayByteSequence(auth)));
   }
 
   /**
@@ -157,7 +85,7 @@
    * @param authorizations authorizations object
    */
   public VisibilityEvaluator(Authorizations authorizations) {
-    this.auths = escape(authorizations);
+    this.accessEvaluator = AccessEvaluator.of(authorizations.toAccessAuthorizations());
   }
 
   /**
@@ -171,42 +99,12 @@
    *         subexpression is of an unknown type
    */
   public boolean evaluate(ColumnVisibility visibility) throws VisibilityParseException {
-    // The VisibilityEvaluator computes a trie from the given Authorizations, that ColumnVisibility
-    // expressions can be evaluated against.
-    return evaluate(visibility.getExpression(), visibility.getParseTree());
-  }
-
-  private final boolean evaluate(final byte[] expression, final Node root)
-      throws VisibilityParseException {
-    if (expression.length == 0) {
-      return true;
-    }
-    switch (root.type) {
-      case TERM:
-        return auths.contains(root.getTerm(expression));
-      case AND:
-        if (root.children == null || root.children.size() < 2) {
-          throw new VisibilityParseException("AND has less than 2 children", expression,
-              root.start);
-        }
-        for (Node child : root.children) {
-          if (!evaluate(expression, child)) {
-            return false;
-          }
-        }
-        return true;
-      case OR:
-        if (root.children == null || root.children.size() < 2) {
-          throw new VisibilityParseException("OR has less than 2 children", expression, root.start);
-        }
-        for (Node child : root.children) {
-          if (evaluate(expression, child)) {
-            return true;
-          }
-        }
-        return false;
-      default:
-        throw new VisibilityParseException("No such node type", expression, root.start);
+    try {
+      return accessEvaluator.canAccess(visibility.getExpression());
+    } catch (InvalidAccessExpressionException e) {
+      // This is thrown for compatability with the exception this class used to evaluate expressions
+      // itself.
+      throw new VisibilityParseException(e);
     }
   }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/security/VisibilityParseException.java b/core/src/main/java/org/apache/accumulo/core/security/VisibilityParseException.java
index 297e575..2d01c2c 100644
--- a/core/src/main/java/org/apache/accumulo/core/security/VisibilityParseException.java
+++ b/core/src/main/java/org/apache/accumulo/core/security/VisibilityParseException.java
@@ -22,6 +22,8 @@
 
 import java.text.ParseException;
 
+import org.apache.accumulo.access.InvalidAccessExpressionException;
+
 /**
  * An exception thrown when a visibility string cannot be parsed.
  */
@@ -41,6 +43,15 @@
     this.visibility = new String(visibility, UTF_8);
   }
 
+  /**
+   * @since 3.1.0
+   */
+  public VisibilityParseException(InvalidAccessExpressionException e) {
+    // TODO need to look at output for this
+    super(e.getDescription(), e.getIndex());
+    this.visibility = e.getPattern();
+  }
+
   @Override
   public String getMessage() {
     return super.getMessage() + " in string '" + visibility + "' at position "
diff --git a/core/src/main/java/org/apache/accumulo/core/singletons/SingletonManager.java b/core/src/main/java/org/apache/accumulo/core/singletons/SingletonManager.java
index 5f3e151..9a38059 100644
--- a/core/src/main/java/org/apache/accumulo/core/singletons/SingletonManager.java
+++ b/core/src/main/java/org/apache/accumulo/core/singletons/SingletonManager.java
@@ -32,8 +32,7 @@
  * Historically, Accumulo client code that used Connector had no control over these singletons. The
  * new AccumuloClient API that replaces Connector is closeable. When all AccumuloClients are closed
  * then resources used by the singletons are released. This class coordinates releasing those
- * resources. For compatibility purposes this class will not release resources when the user has
- * created Connectors.
+ * resources.
  *
  * <p>
  * This class is intermediate solution to resource management. Ideally there would be no static
@@ -62,11 +61,6 @@
      */
     SERVER,
     /**
-     * In this mode singletons are never disabled unless the mode is set back to CLIENT. The user
-     * can do this by using util.CleanUp (an old API created for users).
-     */
-    CONNECTOR,
-    /**
      * In this mode singletons are permanently disabled and entering this mode prevents
      * transitioning to other modes.
      */
@@ -77,7 +71,6 @@
   private static long reservations;
   private static Mode mode;
   private static boolean enabled;
-  private static boolean transitionedFromClientToConnector;
   private static List<SingletonService> services;
 
   @VisibleForTesting
@@ -85,7 +78,6 @@
     reservations = 0;
     mode = Mode.CLIENT;
     enabled = true;
-    transitionedFromClientToConnector = false;
     services = new ArrayList<>();
   }
 
@@ -160,17 +152,6 @@
     if (SingletonManager.mode == Mode.CLOSED) {
       throw new IllegalStateException("Cannot leave closed mode once entered");
     }
-    if (SingletonManager.mode == Mode.CLIENT && mode == Mode.CONNECTOR) {
-      if (transitionedFromClientToConnector) {
-        throw new IllegalStateException("Can only transition from " + Mode.CLIENT + " to "
-            + Mode.CONNECTOR + " once.  This error indicates that "
-            + "org.apache.accumulo.core.util.CleanUp.shutdownNow() was called and then later a "
-            + "Connector was created.  Connectors can not be created after CleanUp.shutdownNow()"
-            + " is called.");
-      }
-
-      transitionedFromClientToConnector = true;
-    }
 
     /*
      * Always allow transition to closed and only allow transition to client/connector when the
@@ -198,10 +179,9 @@
       }
     } else {
       // if we're in a disabled state AND
-      // the mode is CONNECTOR or SERVER or if there are active clients,
+      // the mode is SERVER or if there are active clients,
       // then enable everything
-      if (mode == Mode.CONNECTOR || mode == Mode.SERVER
-          || (mode == Mode.CLIENT && reservations > 0)) {
+      if (mode == Mode.SERVER || (mode == Mode.CLIENT && reservations > 0)) {
         services.forEach(SingletonManager::enable);
         enabled = true;
       }
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/balancer/HostRegexTableLoadBalancer.java b/core/src/main/java/org/apache/accumulo/core/spi/balancer/HostRegexTableLoadBalancer.java
index 0b89e5d..2fd703d 100644
--- a/core/src/main/java/org/apache/accumulo/core/spi/balancer/HostRegexTableLoadBalancer.java
+++ b/core/src/main/java/org/apache/accumulo/core/spi/balancer/HostRegexTableLoadBalancer.java
@@ -19,10 +19,10 @@
 package org.apache.accumulo.core.spi.balancer;
 
 import static java.util.concurrent.TimeUnit.HOURS;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 
 import java.net.InetAddress;
 import java.net.UnknownHostException;
-import java.security.SecureRandom;
 import java.time.Duration;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -62,9 +62,8 @@
 import org.slf4j.LoggerFactory;
 import org.slf4j.event.Level;
 
-import com.google.common.cache.CacheBuilder;
-import com.google.common.cache.CacheLoader;
-import com.google.common.cache.LoadingCache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.LoadingCache;
 import com.google.common.collect.HashMultimap;
 import com.google.common.collect.Multimap;
 
@@ -96,7 +95,6 @@
  */
 public class HostRegexTableLoadBalancer extends TableLoadBalancer {
 
-  private static final SecureRandom random = new SecureRandom();
   private static final String PROP_PREFIX = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey();
 
   private static final Logger LOG = LoggerFactory.getLogger(HostRegexTableLoadBalancer.class);
@@ -267,7 +265,7 @@
   }
 
   private void checkTableConfig(TableId tableId) {
-    Map<String,String> tableRegexes = tablesRegExCache.getUnchecked(tableId).get();
+    Map<String,String> tableRegexes = tablesRegExCache.get(tableId).get();
 
     if (!hrtlbConf.get().regexes.equals(tableRegexes)) {
       LoggerFactory.getLogger(HostRegexTableLoadBalancer.class).warn(
@@ -327,13 +325,8 @@
     this.hrtlbConf = balancerEnvironment.getConfiguration().getDerived(HrtlbConf::new);
 
     tablesRegExCache =
-        CacheBuilder.newBuilder().expireAfterAccess(1, HOURS).build(new CacheLoader<>() {
-          @Override
-          public Supplier<Map<String,String>> load(TableId key) {
-            return balancerEnvironment.getConfiguration(key)
-                .getDerived(HostRegexTableLoadBalancer::getRegexes);
-          }
-        });
+        Caffeine.newBuilder().expireAfterAccess(1, HOURS).build(key -> balancerEnvironment
+            .getConfiguration(key).getDerived(HostRegexTableLoadBalancer::getRegexes));
 
     LOG.info("{}", this);
   }
@@ -427,7 +420,7 @@
                 String poolName = getPoolNameForTable(table);
                 SortedMap<TabletServerId,TServerStatus> currentView = currentGrouped.get(poolName);
                 if (currentView != null) {
-                  int skip = random.nextInt(currentView.size());
+                  int skip = RANDOM.get().nextInt(currentView.size());
                   Iterator<TabletServerId> iter = currentView.keySet().iterator();
                   for (int i = 0; i < skip; i++) {
                     iter.next();
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/balancer/SimpleLoadBalancer.java b/core/src/main/java/org/apache/accumulo/core/spi/balancer/SimpleLoadBalancer.java
index b593a10..d42b7d2 100644
--- a/core/src/main/java/org/apache/accumulo/core/spi/balancer/SimpleLoadBalancer.java
+++ b/core/src/main/java/org/apache/accumulo/core/spi/balancer/SimpleLoadBalancer.java
@@ -57,12 +57,6 @@
  * if possible, but otherwise assignments are made in a random fashion across all available tablet
  * servers.
  *
- * <p>
- * This balancer replaces the deprecated
- * org.apache.accumulo.server.master.balancer.DefaultLoadBalancer which will be removed in a future
- * release. This balancer has the same functionality but uses the stable SPI which does not expose
- * internal types on public methods.
- *
  * @since 2.1.0
  */
 public class SimpleLoadBalancer implements TabletBalancer {
@@ -337,8 +331,10 @@
     TabletId mostRecentlySplit = null;
     long splitTime = 0;
     for (Entry<TabletId,TabletStatistics> entry : extents.entrySet()) {
-      if (entry.getValue().getSplitCreationTime() >= splitTime) {
-        splitTime = entry.getValue().getSplitCreationTime();
+      @SuppressWarnings("deprecation")
+      long splitCreationTime = entry.getValue().getSplitCreationTime();
+      if (splitCreationTime >= splitTime) {
+        splitTime = splitCreationTime;
         mostRecentlySplit = entry.getKey();
       }
     }
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/balancer/data/TabletStatistics.java b/core/src/main/java/org/apache/accumulo/core/spi/balancer/data/TabletStatistics.java
index 99ac7a6..793f00c 100644
--- a/core/src/main/java/org/apache/accumulo/core/spi/balancer/data/TabletStatistics.java
+++ b/core/src/main/java/org/apache/accumulo/core/spi/balancer/data/TabletStatistics.java
@@ -28,6 +28,7 @@
 
   long getNumEntries();
 
+  @Deprecated(since = "3.1")
   long getSplitCreationTime();
 
   double getIngestRate();
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/cache/BlockCache.java b/core/src/main/java/org/apache/accumulo/core/spi/cache/BlockCache.java
index d900149..9ebe3fc 100644
--- a/core/src/main/java/org/apache/accumulo/core/spi/cache/BlockCache.java
+++ b/core/src/main/java/org/apache/accumulo/core/spi/cache/BlockCache.java
@@ -111,5 +111,12 @@
      * @return the number of lookups
      */
     long requestCount();
+
+    /**
+     * @return The number of entries evicted from the cache.
+     *
+     * @since 3.1.0
+     */
+    long evictionCount();
   }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/cache/BlockCacheManager.java b/core/src/main/java/org/apache/accumulo/core/spi/cache/BlockCacheManager.java
index 9af0b6d..7885fbc 100644
--- a/core/src/main/java/org/apache/accumulo/core/spi/cache/BlockCacheManager.java
+++ b/core/src/main/java/org/apache/accumulo/core/spi/cache/BlockCacheManager.java
@@ -21,9 +21,6 @@
 import java.util.HashMap;
 import java.util.Map;
 
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.file.blockfile.cache.impl.BlockCacheConfiguration;
-
 /**
  * @since 2.0.0
  * @see org.apache.accumulo.core.spi
@@ -142,38 +139,4 @@
    */
   protected abstract BlockCache createCache(Configuration conf, CacheType type);
 
-  /**
-   * A convenience method that returns a string of the from
-   * {@code tserver.cache.config.<prefix>.default.} this method is useful for configuring a cache
-   * manager.
-   *
-   * @param prefix A unique identifier that corresponds to a particular BlockCacheManager
-   *        implementation.
-   * @see Configuration#getProperties(String, CacheType)
-   * @deprecated since 2.1.0 because this method does not support scan servers, only tservers. Use
-   *             {@link Configuration#getProperties(String, CacheType)} instead.
-   */
-  @Deprecated(since = "2.1.0")
-  public static String getFullyQualifiedPropertyPrefix(String prefix) {
-    return BlockCacheConfiguration.getFullyQualifiedPropertyPrefix(Property.TSERV_PREFIX, prefix);
-  }
-
-  /**
-   * A convenience method that returns a string of the from
-   * {@code tserver.cache.config.<prefix>.<type>.} this method is useful for configuring a cache
-   * manager.
-   *
-   * @param prefix A unique identifier that corresponds to a particular BlockCacheManager
-   *        implementation.
-   * @see Configuration#getProperties(String, CacheType)
-   *
-   * @deprecated since 2.1.0 because this method does not support scan servers, only tservers. Use
-   *             {@link Configuration#getProperties(String, CacheType)} instead.
-   */
-  @Deprecated(since = "2.1.0")
-  public static String getFullyQualifiedPropertyPrefix(String prefix, CacheType type) {
-    return BlockCacheConfiguration.getFullyQualifiedPropertyPrefix(Property.TSERV_PREFIX, prefix,
-        type);
-  }
-
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/common/ContextClassLoaderFactory.java b/core/src/main/java/org/apache/accumulo/core/spi/common/ContextClassLoaderFactory.java
index 3d9c186..c0e1863 100644
--- a/core/src/main/java/org/apache/accumulo/core/spi/common/ContextClassLoaderFactory.java
+++ b/core/src/main/java/org/apache/accumulo/core/spi/common/ContextClassLoaderFactory.java
@@ -55,16 +55,17 @@
   default void init(ContextClassLoaderEnvironment env) {}
 
   /**
-   * Get the class loader for the given contextName. Callers should not cache the ClassLoader result
-   * as it may change if/when the ClassLoader reloads. Implementations should throw a
-   * RuntimeException of some type (such as IllegalArgumentException) if the provided contextName is
-   * not supported or fails to be constructed.
+   * Get the class loader for the given context. Callers should not cache the ClassLoader result as
+   * it may change if/when the ClassLoader reloads. Implementations should throw a RuntimeException
+   * of some type (such as IllegalArgumentException) if the provided context is not supported or
+   * fails to be constructed.
    *
-   * @param contextName the name of the context that represents a class loader that is managed by
-   *        this factory. Currently, Accumulo will only call this method for non-null and non-empty
+   * @param context the name of the context that represents a class loader that is managed by this
+   *        factory. Currently, Accumulo will only call this method for non-null and non-empty
    *        context. For empty or null context, Accumulo will use the system classloader without
    *        consulting this plugin.
-   * @return the class loader for the given contextName
+   * @return the class loader for the given context
    */
-  ClassLoader getClassLoader(String contextName);
+  ClassLoader getClassLoader(String context);
+
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/compaction/CompactionKind.java b/core/src/main/java/org/apache/accumulo/core/spi/compaction/CompactionKind.java
index 996ca88..32f0dea 100644
--- a/core/src/main/java/org/apache/accumulo/core/spi/compaction/CompactionKind.java
+++ b/core/src/main/java/org/apache/accumulo/core/spi/compaction/CompactionKind.java
@@ -18,6 +18,7 @@
  */
 package org.apache.accumulo.core.spi.compaction;
 
+import org.apache.accumulo.core.client.admin.CompactionConfig;
 import org.apache.accumulo.core.client.admin.compaction.CompactionSelector;
 
 /**
@@ -30,16 +31,20 @@
    */
   SYSTEM,
   /**
-   * Set of files selected by a {@link CompactionSelector} or CompactionStrategy configured for a
-   * table.
+   * Set of files selected by a {@link CompactionSelector} configured for a table.
+   *
+   * @deprecated since 3.1. Use of selector compactions should be replaced with user compactions
+   *             initiated via
+   *             {@link org.apache.accumulo.core.client.admin.TableOperations#compact(String, CompactionConfig)}.
+   *             Everything that can be done with selector compactions can also be done with user
+   *             compactions. User compactions offer more control over when compactions run, the
+   *             range of data compacted, and the ability to cancel. Selector compactions offer none
+   *             of these features and were deprecated in favor of only offering user compactions.
    */
+  @Deprecated(since = "3.1")
   SELECTOR,
   /**
    * A user initiated a one time compaction using an Accumulo client.
    */
-  USER,
-  /**
-   * A compaction executed prior to merging tablets.
-   */
-  CHOP
+  USER
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/compaction/CompactionPlanner.java b/core/src/main/java/org/apache/accumulo/core/spi/compaction/CompactionPlanner.java
index 4b933fc..222ee65 100644
--- a/core/src/main/java/org/apache/accumulo/core/spi/compaction/CompactionPlanner.java
+++ b/core/src/main/java/org/apache/accumulo/core/spi/compaction/CompactionPlanner.java
@@ -45,11 +45,10 @@
 
     /**
      * @return The configured options. For example if the system properties
-     *         {@code tserver.compaction.major.service.s1.planner.opts.p1=abc} and
-     *         {@code tserver.compaction.major.service.s1.planner.opts.p9=123} were set, then this
-     *         map would contain {@code p1=abc} and {@code p9=123}. In this example {@code s1} is
-     *         the identifier for the compaction service. Each compaction service has a single
-     *         planner.
+     *         {@code compaction.service.s1.planner.opts.p1=abc} and
+     *         {@code compaction.service.s1.planner.opts.p9=123} were set, then this map would
+     *         contain {@code p1=abc} and {@code p9=123}. In this example {@code s1} is the
+     *         identifier for the compaction service. Each compaction service has a single planner.
      */
     Map<String,String> getOptions();
 
@@ -57,8 +56,7 @@
      * @return For a given key from the map returned by {@link #getOptions()} determines the fully
      *         qualified tablet property for that key. For example if a planner was being
      *         initialized for compaction service {@code CS9} and this method were passed
-     *         {@code prop1} then it would return
-     *         {@code tserver.compaction.major.service.CS9.planner.opts.prop1}.
+     *         {@code prop1} then it would return {@code compaction.service.CS9.planner.opts.prop1}.
      */
     String getFullyQualifiedOption(String key);
 
@@ -141,27 +139,9 @@
    * the candidates will contain the files it did not compact and the results of any previous
    * compactions it scheduled. The planner must eventually compact all of the files in the candidate
    * set down to a single file. The compaction service will keep calling the planner until it does.
-   * <li>CompactionKind.CHOP. The planner is required to eventually compact all candidates. One
-   * major difference with USER compactions is this kind is not required to compact all files to a
-   * single file. It is ok to return a compaction plan that compacts a subset of the candidates.
-   * When the planner compacts a subset, it will eventually be called later. When it is called later
-   * the candidates will contain the files it did not compact.
    * </ul>
    *
    * <p>
-   * For a chop compaction assume the following happens.
-   * <ol>
-   * <li>The candidate set passed to makePlan contains the files {@code [F1,F2,F3,F4]} and kind is
-   * CHOP
-   * <li>The planner returns a job to compact files {@code [F1,F2]} on executor E1
-   * <li>The compaction runs compacting {@code [F1,F2]} into file {@code [F5]}
-   * </ol>
-   *
-   * <p>
-   * For the case above, eventually the planner will called again with a candidate set of
-   * {@code [F3,F4]} and it must eventually compact those two files.
-   *
-   * <p>
    * For a user and selector compaction assume the same thing happens, it will result in a slightly
    * different outcome.
    * <ol>
@@ -173,9 +153,7 @@
    *
    * <p>
    * For the case above, eventually the planner will called again with a candidate set of
-   * {@code [F3,F4,F5]} and it must eventually compact those three files to one. The difference with
-   * CHOP compactions is that the result of intermediate compactions are included in the candidate
-   * set.
+   * {@code [F3,F4,F5]} and it must eventually compact those three files to one.
    *
    * <p>
    * When a planner returns a compactions plan, task will be queued on executors. Previously queued
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/compaction/DefaultCompactionPlanner.java b/core/src/main/java/org/apache/accumulo/core/spi/compaction/DefaultCompactionPlanner.java
index 73952b7..ee07bc2 100644
--- a/core/src/main/java/org/apache/accumulo/core/spi/compaction/DefaultCompactionPlanner.java
+++ b/core/src/main/java/org/apache/accumulo/core/spi/compaction/DefaultCompactionPlanner.java
@@ -18,9 +18,13 @@
  */
 package org.apache.accumulo.core.spi.compaction;
 
+import static org.apache.accumulo.core.util.LazySingletons.GSON;
+
+import java.lang.reflect.Field;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
@@ -28,17 +32,22 @@
 import java.util.List;
 import java.util.Objects;
 import java.util.Set;
+import java.util.stream.Collectors;
 
 import org.apache.accumulo.core.client.admin.compaction.CompactableFile;
 import org.apache.accumulo.core.conf.ConfigurationTypeHelper;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.spi.common.ServiceEnvironment;
 import org.apache.accumulo.core.util.compaction.CompactionJobPrioritizer;
+import org.apache.accumulo.core.util.compaction.DeprecatedCompactionKind;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Preconditions;
-import com.google.gson.Gson;
+import com.google.gson.JsonArray;
+import com.google.gson.JsonElement;
+import com.google.gson.JsonObject;
+import com.google.gson.JsonParseException;
 
 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 
@@ -51,8 +60,8 @@
  * compaction service you are configuring.
  *
  * <ul>
- * <li>{@code tserver.compaction.major.service.<service>.opts.executors} This is a json array of
- * objects where each object has the fields:
+ * <li>{@code compaction.service.<service>.opts.executors} This is a json array of objects where
+ * each object has the fields:
  * <table>
  * <caption>Default Compaction Planner Executor options</caption>
  * <tr>
@@ -83,16 +92,19 @@
  * </tr>
  * </table>
  * <br>
- * The maxSize field determines the maximum size of compaction that will run on an executor. The
- * maxSize field can have a suffix of K,M,G for kilobytes, megabytes, or gigabytes and represents
- * the sum of the input files for a given compaction. One executor can have no max size and it will
- * run everything that is too large for the other executors. If all executors have a max size, then
- * system compactions will only run for compactions smaller than the largest max size. User, chop,
- * and selector compactions will always run, even if there is no executor for their size. These
- * compactions will run on the executor with the largest max size. The following example value for
- * this property will create 3 threads to run compactions of files whose file size sum is less than
- * 100M, 3 threads to run compactions of files whose file size sum is less than 500M, and run all
- * other compactions on Compactors configured to run compactions for Queue1:
+ * Note: The "executors" option has been deprecated in 3.1 and will be removed in a future release.
+ * This example uses the new `compaction.service` prefix. The property prefix
+ * "tserver.compaction.major.service" has also been deprecated in 3.1 and will be removed in a
+ * future release. The maxSize field determines the maximum size of compaction that will run on an
+ * executor. The maxSize field can have a suffix of K,M,G for kilobytes, megabytes, or gigabytes and
+ * represents the sum of the input files for a given compaction. One executor can have no max size
+ * and it will run everything that is too large for the other executors. If all executors have a max
+ * size, then system compactions will only run for compactions smaller than the largest max size.
+ * User, chop, and selector compactions will always run, even if there is no executor for their
+ * size. These compactions will run on the executor with the largest max size. The following example
+ * value for this property will create 3 threads to run compactions of files whose file size sum is
+ * less than 100M, 3 threads to run compactions of files whose file size sum is less than 500M, and
+ * run all other compactions on Compactors configured to run compactions for Queue1:
  *
  * <pre>
  * {@code
@@ -105,8 +117,28 @@
  *
  * Note that the use of 'external' requires that the CompactionCoordinator and at least one
  * Compactor for Queue1 is running.
- * <li>{@code tserver.compaction.major.service.<service>.opts.maxOpen} This determines the maximum
- * number of files that will be included in a single compaction.
+ * <li>{@code compaction.service.<service>.opts.maxOpen} This determines the maximum number of files
+ * that will be included in a single compaction.
+ * <li>{@code compaction.service.<service>.opts.queues} This is a json array of queue objects which
+ * have the following fields:
+ * <table>
+ * <caption>Default Compaction Planner Queue options</caption>
+ * <tr>
+ * <th>Field Name</th>
+ * <th>Description</th>
+ * </tr>
+ * <tr>
+ * <td>name</td>
+ * <td>name or alias of the queue (required)</td>
+ * </tr>
+ * <tr>
+ * <td>maxSize</td>
+ * <td>threshold sum of the input files (required for all but one of the configs)</td>
+ * </tr>
+ * </table>
+ * <br>
+ * This 'queues' object is used for defining external compaction queues without needing to use the
+ * thread-based 'executors' property.
  * </ul>
  *
  * <p>
@@ -124,13 +156,13 @@
  * ratio less than 3 that results in a compaction.
  *
  *
- * @since 2.1.0
+ * @since 3.1.0
  * @see org.apache.accumulo.core.spi.compaction
  */
 
 public class DefaultCompactionPlanner implements CompactionPlanner {
 
-  private static final Logger log = LoggerFactory.getLogger(DefaultCompactionPlanner.class);
+  private final static Logger log = LoggerFactory.getLogger(DefaultCompactionPlanner.class);
 
   private static class ExecutorConfig {
     String type;
@@ -140,6 +172,11 @@
     String queue;
   }
 
+  private static class QueueConfig {
+    String name;
+    String maxSize;
+  }
+
   private static class Executor {
     final CompactionExecutorId ceid;
     final Long maxSize;
@@ -167,19 +204,23 @@
       justification = "Field is written by Gson")
   @Override
   public void init(InitParameters params) {
+    List<Executor> tmpExec = new ArrayList<>();
+    String values;
 
     if (params.getOptions().containsKey("executors")
         && !params.getOptions().get("executors").isBlank()) {
+      values = params.getOptions().get("executors");
 
-      ExecutorConfig[] execConfigs =
-          new Gson().fromJson(params.getOptions().get("executors"), ExecutorConfig[].class);
+      // Generate a list of fields from the desired object.
+      final List<String> execFields = Arrays.stream(ExecutorConfig.class.getDeclaredFields())
+          .map(Field::getName).collect(Collectors.toList());
 
-      List<Executor> tmpExec = new ArrayList<>();
+      for (JsonElement element : GSON.get().fromJson(values, JsonArray.class)) {
+        validateConfig(element, execFields, ExecutorConfig.class.getName());
+        ExecutorConfig executorConfig = GSON.get().fromJson(element, ExecutorConfig.class);
 
-      for (ExecutorConfig executorConfig : execConfigs) {
         Long maxSize = executorConfig.maxSize == null ? null
             : ConfigurationTypeHelper.getFixedMemoryAsBytes(executorConfig.maxSize);
-
         CompactionExecutorId ceid;
 
         // If not supplied, GSON will leave type null. Default to internal
@@ -207,143 +248,169 @@
         }
         tmpExec.add(new Executor(ceid, maxSize));
       }
-
-      Collections.sort(tmpExec, Comparator.comparing(Executor::getMaxSize,
-          Comparator.nullsLast(Comparator.naturalOrder())));
-
-      executors = List.copyOf(tmpExec);
-
-      if (executors.stream().filter(e -> e.getMaxSize() == null).count() > 1) {
-        throw new IllegalArgumentException(
-            "Can only have one executor w/o a maxSize. " + params.getOptions().get("executors"));
-      }
-
-      // use the add method on the Set interface to check for duplicate maxSizes
-      Set<Long> maxSizes = new HashSet<>();
-      executors.forEach(e -> {
-        if (!maxSizes.add(e.getMaxSize())) {
-          throw new IllegalArgumentException(
-              "Duplicate maxSize set in executors. " + params.getOptions().get("executors"));
-        }
-      });
-    } else {
-      throw new IllegalStateException("No defined executors for this planner");
     }
+
+    if (params.getOptions().containsKey("queues") && !params.getOptions().get("queues").isBlank()) {
+      values = params.getOptions().get("queues");
+
+      // Generate a list of fields from the desired object.
+      final List<String> queueFields = Arrays.stream(QueueConfig.class.getDeclaredFields())
+          .map(Field::getName).collect(Collectors.toList());
+
+      for (JsonElement element : GSON.get().fromJson(values, JsonArray.class)) {
+        validateConfig(element, queueFields, QueueConfig.class.getName());
+        QueueConfig queueConfig = GSON.get().fromJson(element, QueueConfig.class);
+
+        Long maxSize = queueConfig.maxSize == null ? null
+            : ConfigurationTypeHelper.getFixedMemoryAsBytes(queueConfig.maxSize);
+
+        CompactionExecutorId ceid;
+        String queue = Objects.requireNonNull(queueConfig.name, "'name' must be specified");
+        ceid = params.getExecutorManager().getExternalExecutor(queue);
+        tmpExec.add(new Executor(ceid, maxSize));
+      }
+    }
+
+    if (tmpExec.size() < 1) {
+      throw new IllegalStateException("No defined executors or queues for this planner");
+    }
+
+    tmpExec.sort(Comparator.comparing(Executor::getMaxSize,
+        Comparator.nullsLast(Comparator.naturalOrder())));
+
+    executors = List.copyOf(tmpExec);
+
+    if (executors.stream().filter(e -> e.getMaxSize() == null).count() > 1) {
+      throw new IllegalArgumentException(
+          "Can only have one executor w/o a maxSize. " + params.getOptions().get("executors"));
+    }
+
+    // use the add method on the Set interface to check for duplicate maxSizes
+    Set<Long> maxSizes = new HashSet<>();
+    executors.forEach(e -> {
+      if (!maxSizes.add(e.getMaxSize())) {
+        throw new IllegalArgumentException(
+            "Duplicate maxSize set in executors. " + params.getOptions().get("executors"));
+      }
+    });
+
     determineMaxFilesToCompact(params);
   }
 
-  @SuppressWarnings("removal")
+  @SuppressWarnings("deprecation")
   private void determineMaxFilesToCompact(InitParameters params) {
-    String fqo = params.getFullyQualifiedOption("maxOpen");
-    if (!params.getServiceEnvironment().getConfiguration().isSet(fqo)
-        && params.getServiceEnvironment().getConfiguration()
-            .isSet(Property.TSERV_MAJC_THREAD_MAXOPEN.getKey())) {
-      log.warn("The property " + Property.TSERV_MAJC_THREAD_MAXOPEN.getKey()
-          + " was set, it is deprecated.  Set the " + fqo + " option instead.");
-      this.maxFilesToCompact = Integer.parseInt(params.getServiceEnvironment().getConfiguration()
-          .get(Property.TSERV_MAJC_THREAD_MAXOPEN.getKey()));
-    } else {
-      this.maxFilesToCompact = Integer.parseInt(params.getOptions().getOrDefault("maxOpen",
-          Property.TSERV_COMPACTION_SERVICE_DEFAULT_MAX_OPEN.getDefaultValue()));
+
+    String maxOpen = params.getOptions().get("maxOpen");
+    if (maxOpen == null) {
+      maxOpen = Property.TSERV_COMPACTION_SERVICE_DEFAULT_MAX_OPEN.getDefaultValue();
+      log.trace("default maxOpen not set, defaulting to {}", maxOpen);
+    }
+    this.maxFilesToCompact = Integer.parseInt(maxOpen);
+  }
+
+  private void validateConfig(JsonElement json, List<String> fields, String className) {
+
+    JsonObject jsonObject = GSON.get().fromJson(json, JsonObject.class);
+
+    List<String> objectProperties = new ArrayList<>(jsonObject.keySet());
+    HashSet<String> classFieldNames = new HashSet<>(fields);
+
+    if (!classFieldNames.containsAll(objectProperties)) {
+      objectProperties.removeAll(classFieldNames);
+      throw new JsonParseException(
+          "Invalid fields: " + objectProperties + " provided for class: " + className);
     }
   }
 
   @Override
   public CompactionPlan makePlan(PlanningParameters params) {
-    try {
+    if (params.getCandidates().isEmpty()) {
+      return params.createPlanBuilder().build();
+    }
 
-      if (params.getCandidates().isEmpty()) {
-        return params.createPlanBuilder().build();
+    Set<CompactableFile> filesCopy = new HashSet<>(params.getCandidates());
+
+    long maxSizeToCompact = getMaxSizeToCompact(params.getKind());
+
+    Collection<CompactableFile> group;
+    if (params.getRunningCompactions().isEmpty()) {
+      group =
+          findDataFilesToCompact(filesCopy, params.getRatio(), maxFilesToCompact, maxSizeToCompact);
+
+      if (!group.isEmpty() && group.size() < params.getCandidates().size()
+          && params.getCandidates().size() <= maxFilesToCompact
+          && (params.getKind() == CompactionKind.USER
+              || params.getKind() == DeprecatedCompactionKind.SELECTOR)) {
+        // USER and SELECTOR compactions must eventually compact all files. When a subset of files
+        // that meets the compaction ratio is selected, look ahead and see if the next compaction
+        // would also meet the compaction ratio. If not then compact everything to avoid doing
+        // more than logarithmic work across multiple comapctions.
+
+        filesCopy.removeAll(group);
+        filesCopy.add(getExpected(group, 0));
+
+        if (findDataFilesToCompact(filesCopy, params.getRatio(), maxFilesToCompact,
+            maxSizeToCompact).isEmpty()) {
+          // The next possible compaction does not meet the compaction ratio, so compact
+          // everything.
+          group = Set.copyOf(params.getCandidates());
+        }
+
       }
 
-      Set<CompactableFile> filesCopy = new HashSet<>(params.getCandidates());
+    } else if (params.getKind() == CompactionKind.SYSTEM) {
+      // This code determines if once the files compacting finish would they be included in a
+      // compaction with the files smaller than them? If so, then wait for the running compaction
+      // to complete.
 
-      long maxSizeToCompact = getMaxSizeToCompact(params.getKind());
+      // The set of files running compactions may produce
+      var expectedFiles = getExpected(params.getRunningCompactions());
 
-      Collection<CompactableFile> group;
-      if (params.getRunningCompactions().isEmpty()) {
-        group = findDataFilesToCompact(filesCopy, params.getRatio(), maxFilesToCompact,
-            maxSizeToCompact);
+      if (!Collections.disjoint(filesCopy, expectedFiles)) {
+        throw new AssertionError();
+      }
 
-        if (!group.isEmpty() && group.size() < params.getCandidates().size()
-            && params.getCandidates().size() <= maxFilesToCompact
-            && (params.getKind() == CompactionKind.USER
-                || params.getKind() == CompactionKind.SELECTOR)) {
-          // USER and SELECTOR compactions must eventually compact all files. When a subset of files
-          // that meets the compaction ratio is selected, look ahead and see if the next compaction
-          // would also meet the compaction ratio. If not then compact everything to avoid doing
-          // more than logarithmic work across multiple comapctions.
+      filesCopy.addAll(expectedFiles);
 
-          filesCopy.removeAll(group);
-          filesCopy.add(getExpected(group, 0));
+      group =
+          findDataFilesToCompact(filesCopy, params.getRatio(), maxFilesToCompact, maxSizeToCompact);
 
-          if (findDataFilesToCompact(filesCopy, params.getRatio(), maxFilesToCompact,
-              maxSizeToCompact).isEmpty()) {
-            // The next possible compaction does not meet the compaction ratio, so compact
-            // everything.
-            group = Set.copyOf(params.getCandidates());
-          }
-
-        }
-
-      } else if (params.getKind() == CompactionKind.SYSTEM) {
-        // This code determines if once the files compacting finish would they be included in a
-        // compaction with the files smaller than them? If so, then wait for the running compaction
-        // to complete.
-
-        // The set of files running compactions may produce
-        var expectedFiles = getExpected(params.getRunningCompactions());
-
-        if (!Collections.disjoint(filesCopy, expectedFiles)) {
-          throw new AssertionError();
-        }
-
-        filesCopy.addAll(expectedFiles);
-
-        group = findDataFilesToCompact(filesCopy, params.getRatio(), maxFilesToCompact,
-            maxSizeToCompact);
-
-        if (!Collections.disjoint(group, expectedFiles)) {
-          // file produced by running compaction will eventually compact with existing files, so
-          // wait.
-          group = Set.of();
-        }
-      } else {
+      if (!Collections.disjoint(group, expectedFiles)) {
+        // file produced by running compaction will eventually compact with existing files, so
+        // wait.
         group = Set.of();
       }
+    } else {
+      group = Set.of();
+    }
 
-      if (group.isEmpty()) {
-
-        if ((params.getKind() == CompactionKind.USER || params.getKind() == CompactionKind.SELECTOR
-            || params.getKind() == CompactionKind.CHOP)
-            && params.getRunningCompactions().stream()
-                .noneMatch(job -> job.getKind() == params.getKind())) {
-          group = findMaximalRequiredSetToCompact(params.getCandidates(), maxFilesToCompact);
-        } else if (params.getKind() == CompactionKind.SYSTEM
-            && params.getRunningCompactions().isEmpty()
-            && params.getAll().size() == params.getCandidates().size()) {
-          int maxTabletFiles = getMaxTabletFiles(
-              params.getServiceEnvironment().getConfiguration(params.getTableId()));
-          if (params.getAll().size() > maxTabletFiles) {
-            // The tablet is above its max files, there are no compactions running, all files are
-            // candidates for a system compaction, and no files were found to compact. Attempt to
-            // find a set of files to compact by lowering the compaction ratio.
-            group = findFilesToCompactWithLowerRatio(params, maxSizeToCompact, maxTabletFiles);
-          }
+    if (group.isEmpty()) {
+      if ((params.getKind() == CompactionKind.USER
+          || params.getKind() == DeprecatedCompactionKind.SELECTOR)
+          && params.getRunningCompactions().stream()
+              .noneMatch(job -> job.getKind() == params.getKind())) {
+        group = findMaximalRequiredSetToCompact(params.getCandidates(), maxFilesToCompact);
+      } else if (params.getKind() == CompactionKind.SYSTEM
+          && params.getRunningCompactions().isEmpty()
+          && params.getAll().size() == params.getCandidates().size()) {
+        int maxTabletFiles =
+            getMaxTabletFiles(params.getServiceEnvironment().getConfiguration(params.getTableId()));
+        if (params.getAll().size() > maxTabletFiles) {
+          // The tablet is above its max files, there are no compactions running, all files are
+          // candidates for a system compaction, and no files were found to compact. Attempt to
+          // find a set of files to compact by lowering the compaction ratio.
+          group = findFilesToCompactWithLowerRatio(params, maxSizeToCompact, maxTabletFiles);
         }
       }
+    }
 
-      if (group.isEmpty()) {
-        return params.createPlanBuilder().build();
-      } else {
-        // determine which executor to use based on the size of the files
-        var ceid = getExecutor(group);
+    if (group.isEmpty()) {
+      return params.createPlanBuilder().build();
+    } else {
+      // determine which executor to use based on the size of the files
+      var ceid = getExecutor(group);
 
-        return params.createPlanBuilder().addJob(createPriority(params, group), ceid, group)
-            .build();
-      }
-    } catch (RuntimeException e) {
-      throw e;
+      return params.createPlanBuilder().addJob(createPriority(params, group), ceid, group).build();
     }
   }
 
@@ -438,7 +505,7 @@
           new URI("hdfs://fake/accumulo/tables/adef/t-zzFAKEzz/FAKE-0000" + count + ".rf"), size,
           0);
     } catch (URISyntaxException e) {
-      throw new RuntimeException(e);
+      throw new IllegalStateException(e);
     }
   }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/compaction/package-info.java b/core/src/main/java/org/apache/accumulo/core/spi/compaction/package-info.java
index fd7d094..e1bea0f 100644
--- a/core/src/main/java/org/apache/accumulo/core/spi/compaction/package-info.java
+++ b/core/src/main/java/org/apache/accumulo/core/spi/compaction/package-info.java
@@ -65,9 +65,6 @@
  * to periodically select files to compact. This supports use cases like periodically compacting all
  * files because there are too many deletes. See
  * {@link org.apache.accumulo.core.client.admin.compaction.CompactionSelector}
- * <li><b>Compaction Strategy</b> A deprecated pluggable component replaced by the Selector and
- * Configurer. See {@code org.apache.accumulo.core.client.admin.CompactionStrategyConfig}'s own
- * documentation for more information about why it was deprecated.
  * </ul>
  * </ul>
  * </ul>
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/crypto/AESCryptoService.java b/core/src/main/java/org/apache/accumulo/core/spi/crypto/AESCryptoService.java
index 15e5f45..4ed55ac 100644
--- a/core/src/main/java/org/apache/accumulo/core/spi/crypto/AESCryptoService.java
+++ b/core/src/main/java/org/apache/accumulo/core/spi/crypto/AESCryptoService.java
@@ -19,6 +19,7 @@
 package org.apache.accumulo.core.spi.crypto;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
@@ -78,7 +79,6 @@
   private static final String NO_CRYPTO_VERSION = "U+1F47B";
   private static final String URI = "uri";
   private static final String KEY_WRAP_TRANSFORM = "AESWrap";
-  private static final SecureRandom random = new SecureRandom();
 
   private Key encryptingKek = null;
   private String keyLocation = null;
@@ -358,8 +358,8 @@
         } catch (NoSuchAlgorithmException | NoSuchPaddingException e) {
           throw new CryptoException("Error obtaining cipher for transform " + transformation, e);
         }
-        this.fek = generateKey(random, KEY_LENGTH_IN_BYTES);
-        random.nextBytes(this.initVector);
+        this.fek = generateKey(RANDOM.get(), KEY_LENGTH_IN_BYTES);
+        RANDOM.get().nextBytes(this.initVector);
         this.firstInitVector = Arrays.copyOf(this.initVector, this.initVector.length);
         this.decryptionParameters =
             createCryptoParameters(VERSION, encryptingKek, keyLocation, keyManager, fek);
@@ -499,14 +499,14 @@
         } catch (NoSuchAlgorithmException | NoSuchPaddingException e) {
           throw new CryptoException("Error obtaining cipher for transform " + transformation, e);
         }
-        this.fek = generateKey(random, KEY_LENGTH_IN_BYTES);
+        this.fek = generateKey(RANDOM.get(), KEY_LENGTH_IN_BYTES);
         this.decryptionParameters =
             createCryptoParameters(VERSION, encryptingKek, keyLocation, keyManager, fek);
       }
 
       @Override
       public OutputStream encryptStream(OutputStream outputStream) throws CryptoException {
-        random.nextBytes(initVector);
+        RANDOM.get().nextBytes(initVector);
         try {
           outputStream.write(initVector);
         } catch (IOException e) {
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/fs/DelegatingChooser.java b/core/src/main/java/org/apache/accumulo/core/spi/fs/DelegatingChooser.java
index e76a1b7..eb39ada 100644
--- a/core/src/main/java/org/apache/accumulo/core/spi/fs/DelegatingChooser.java
+++ b/core/src/main/java/org/apache/accumulo/core/spi/fs/DelegatingChooser.java
@@ -89,7 +89,7 @@
           + Property.GENERAL_ARBITRARY_PROP_PREFIX + DEFAULT_SCOPED_VOLUME_CHOOSER
           + " must be a valid " + VolumeChooser.class.getSimpleName() + " to use the "
           + getClass().getSimpleName();
-      throw new RuntimeException(msg);
+      throw new IllegalStateException(msg);
     }
 
     return createVolumeChooser(env, clazz, TABLE_CUSTOM_SUFFIX, env.getTable().orElseThrow(),
@@ -115,7 +115,7 @@
             + Property.GENERAL_ARBITRARY_PROP_PREFIX + DEFAULT_SCOPED_VOLUME_CHOOSER
             + " must be a valid " + VolumeChooser.class.getSimpleName() + " to use the "
             + getClass().getSimpleName();
-        throw new RuntimeException(msg);
+        throw new IllegalStateException(msg);
       }
 
       property = DEFAULT_SCOPED_VOLUME_CHOOSER;
@@ -155,10 +155,10 @@
         } else {
           return env.getServiceEnv().instantiate(className, VolumeChooser.class);
         }
-      } catch (Exception e) {
+      } catch (ReflectiveOperationException e) {
         String msg = "Failed to create instance for " + key + " configured to use " + className
             + " via " + property;
-        throw new RuntimeException(msg, e);
+        throw new IllegalStateException(msg, e);
       }
     });
   }
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/fs/PreferredVolumeChooser.java b/core/src/main/java/org/apache/accumulo/core/spi/fs/PreferredVolumeChooser.java
index 22c57ce..e941910 100644
--- a/core/src/main/java/org/apache/accumulo/core/spi/fs/PreferredVolumeChooser.java
+++ b/core/src/main/java/org/apache/accumulo/core/spi/fs/PreferredVolumeChooser.java
@@ -182,7 +182,7 @@
     if (preferredVolumes == null || preferredVolumes.isEmpty()) {
       String msg = "Property " + TABLE_CUSTOM_SUFFIX + " or " + DEFAULT_SCOPED_PREFERRED_VOLUMES
           + " must be a subset of " + options + " to use the " + getClass().getSimpleName();
-      throw new RuntimeException(msg);
+      throw new IllegalArgumentException(msg);
     }
 
     return parsePreferred(TABLE_CUSTOM_SUFFIX, preferredVolumes, options);
@@ -208,7 +208,7 @@
       if (preferredVolumes == null || preferredVolumes.isEmpty()) {
         String msg = "Property " + property + " or " + DEFAULT_SCOPED_PREFERRED_VOLUMES
             + " must be a subset of " + options + " to use the " + getClass().getSimpleName();
-        throw new RuntimeException(msg);
+        throw new IllegalArgumentException(msg);
       }
 
       property = DEFAULT_SCOPED_PREFERRED_VOLUMES;
@@ -226,13 +226,13 @@
     if (preferred.isEmpty()) {
       String msg = "No volumes could be parsed from '" + property + "', which had a value of '"
           + preferredVolumes + "'";
-      throw new RuntimeException(msg);
+      throw new IllegalArgumentException(msg);
     }
     // preferred volumes should also exist in the original options (typically, from
     // instance.volumes)
     if (Collections.disjoint(preferred, options)) {
       String msg = "Some volumes in " + preferred + " are not valid volumes from " + options;
-      throw new RuntimeException(msg);
+      throw new IllegalArgumentException(msg);
     }
 
     return preferred;
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/fs/RandomVolumeChooser.java b/core/src/main/java/org/apache/accumulo/core/spi/fs/RandomVolumeChooser.java
index 7754991..55f1781 100644
--- a/core/src/main/java/org/apache/accumulo/core/spi/fs/RandomVolumeChooser.java
+++ b/core/src/main/java/org/apache/accumulo/core/spi/fs/RandomVolumeChooser.java
@@ -18,7 +18,8 @@
  */
 package org.apache.accumulo.core.spi.fs;
 
-import java.security.SecureRandom;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
+
 import java.util.Set;
 
 /**
@@ -27,7 +28,6 @@
  * @since 2.1.0
  */
 public class RandomVolumeChooser implements VolumeChooser {
-  private static final SecureRandom random = new SecureRandom();
 
   /**
    * Selects a volume at random from the provided set of volumes. The environment scope is not
@@ -36,7 +36,7 @@
   @Override
   public String choose(VolumeChooserEnvironment env, Set<String> options) {
     String[] optionsArray = options.toArray(new String[0]);
-    return optionsArray[random.nextInt(optionsArray.length)];
+    return optionsArray[RANDOM.get().nextInt(optionsArray.length)];
   }
 
   /**
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/fs/SpaceAwareVolumeChooser.java b/core/src/main/java/org/apache/accumulo/core/spi/fs/SpaceAwareVolumeChooser.java
index 20b92e0..99a29db 100644
--- a/core/src/main/java/org/apache/accumulo/core/spi/fs/SpaceAwareVolumeChooser.java
+++ b/core/src/main/java/org/apache/accumulo/core/spi/fs/SpaceAwareVolumeChooser.java
@@ -19,13 +19,12 @@
 package org.apache.accumulo.core.spi.fs;
 
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 
 import java.io.IOException;
-import java.security.SecureRandom;
 import java.util.NavigableMap;
 import java.util.Set;
 import java.util.TreeMap;
-import java.util.concurrent.ExecutionException;
 
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.conf.Configuration;
@@ -35,9 +34,8 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.cache.CacheBuilder;
-import com.google.common.cache.CacheLoader;
-import com.google.common.cache.LoadingCache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.LoadingCache;
 
 /**
  * A {@link PreferredVolumeChooser} that takes remaining HDFS space into account when making a
@@ -48,8 +46,6 @@
  */
 public class SpaceAwareVolumeChooser extends PreferredVolumeChooser {
 
-  private static final SecureRandom random = new SecureRandom();
-
   public static final String RECOMPUTE_INTERVAL = "spaceaware.volume.chooser.recompute.interval";
 
   // Default time to wait in ms. Defaults to 5 min
@@ -68,11 +64,7 @@
 
   @Override
   public String choose(VolumeChooserEnvironment env, Set<String> options) {
-    try {
-      return getCache(env).get(getPreferredVolumes(env, options)).next();
-    } catch (ExecutionException e) {
-      throw new IllegalStateException("Execution exception when attempting to cache choice", e);
-    }
+    return getCache(env).get(getPreferredVolumes(env, options)).next();
   }
 
   private synchronized LoadingCache<Set<String>,WeightedRandomCollection>
@@ -84,13 +76,8 @@
       long computationCacheDuration = StringUtils.isNotBlank(propertyValue)
           ? Long.parseLong(propertyValue) : defaultComputationCacheDuration;
 
-      choiceCache = CacheBuilder.newBuilder()
-          .expireAfterWrite(computationCacheDuration, MILLISECONDS).build(new CacheLoader<>() {
-            @Override
-            public WeightedRandomCollection load(Set<String> key) {
-              return new WeightedRandomCollection(key, env);
-            }
-          });
+      choiceCache = Caffeine.newBuilder().expireAfterWrite(computationCacheDuration, MILLISECONDS)
+          .build(key -> new WeightedRandomCollection(key, env));
     }
 
     return choiceCache;
@@ -134,7 +121,7 @@
     }
 
     public String next() {
-      double value = random.nextDouble() * total;
+      double value = RANDOM.get().nextDouble() * total;
       return map.higherEntry(value).getValue();
     }
   }
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/scan/ConfigurableScanServerHostSelector.java b/core/src/main/java/org/apache/accumulo/core/spi/scan/ConfigurableScanServerHostSelector.java
index d21a879..f43f21e 100644
--- a/core/src/main/java/org/apache/accumulo/core/spi/scan/ConfigurableScanServerHostSelector.java
+++ b/core/src/main/java/org/apache/accumulo/core/spi/scan/ConfigurableScanServerHostSelector.java
@@ -18,6 +18,8 @@
  */
 package org.apache.accumulo.core.spi.scan;
 
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
+
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Comparator;
@@ -139,7 +141,7 @@
         } else {
           final HashCode hashCode = hashTablet(tablet, profile.getSalt(numberOfPreviousAttempts));
           final int serverIndex =
-              (Math.abs(hashCode.asInt()) + RANDOM.nextInt(numServersToUseInAttemptPlan))
+              (Math.abs(hashCode.asInt()) + RANDOM.get().nextInt(numServersToUseInAttemptPlan))
                   % hostIndex.size();
           final String hostToUse = hostIndex.get(serverIndex);
           final List<String> scanServersOnHost = scanServerHosts.get(hostToUse);
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/scan/ConfigurableScanServerSelector.java b/core/src/main/java/org/apache/accumulo/core/spi/scan/ConfigurableScanServerSelector.java
index e0b9ddb..a7900645 100644
--- a/core/src/main/java/org/apache/accumulo/core/spi/scan/ConfigurableScanServerSelector.java
+++ b/core/src/main/java/org/apache/accumulo/core/spi/scan/ConfigurableScanServerSelector.java
@@ -19,9 +19,10 @@
 package org.apache.accumulo.core.spi.scan;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.core.util.LazySingletons.GSON;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 
 import java.lang.reflect.Type;
-import java.security.SecureRandom;
 import java.time.Duration;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -42,7 +43,6 @@
 import com.google.common.collect.Sets;
 import com.google.common.hash.HashCode;
 import com.google.common.hash.Hashing;
-import com.google.gson.Gson;
 import com.google.gson.reflect.TypeToken;
 
 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
@@ -174,7 +174,6 @@
  */
 public class ConfigurableScanServerSelector implements ScanServerSelector {
 
-  protected static final SecureRandom RANDOM = new SecureRandom();
   public static final String PROFILES_DEFAULT = "[{'isDefault':true,'maxBusyTimeout':'5m',"
       + "'busyTimeoutMultiplier':8, 'scanTypeActivations':[], "
       + "'attemptPlans':[{'servers':'3', 'busyTimeout':'33ms', 'salt':'one'},"
@@ -298,9 +297,8 @@
 
   private void parseProfiles(Map<String,String> options) {
     Type listType = new TypeToken<ArrayList<Profile>>() {}.getType();
-    Gson gson = new Gson();
     List<Profile> profList =
-        gson.fromJson(options.getOrDefault("profiles", PROFILES_DEFAULT), listType);
+        GSON.get().fromJson(options.getOrDefault("profiles", PROFILES_DEFAULT), listType);
 
     profiles = new HashMap<>();
     defaultProfile = null;
@@ -436,8 +434,8 @@
 
       var hashCode = hashTablet(tablet, profile.getSalt(attempts));
 
-      int serverIndex =
-          (Math.abs(hashCode.asInt()) + RANDOM.nextInt(numServers)) % orderedScanServers.size();
+      int serverIndex = (Math.abs(hashCode.asInt()) + RANDOM.get().nextInt(numServers))
+          % orderedScanServers.size();
 
       serverToUse = orderedScanServers.get(serverIndex);
 
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/scan/ScanDispatcher.java b/core/src/main/java/org/apache/accumulo/core/spi/scan/ScanDispatcher.java
index c6f362e..19bcbe8 100644
--- a/core/src/main/java/org/apache/accumulo/core/spi/scan/ScanDispatcher.java
+++ b/core/src/main/java/org/apache/accumulo/core/spi/scan/ScanDispatcher.java
@@ -64,31 +64,6 @@
   }
 
   /**
-   * The method parameters for {@link ScanDispatcher#dispatch(DispatchParmaters)}. This interface
-   * exists so the API can evolve and additional parameters can be passed to the method in the
-   * future.
-   *
-   * @since 2.0.0
-   * @deprecated since 2.1.0 replaced by {@link DispatchParameters} and
-   *             {@link ScanDispatcher#dispatch(DispatchParameters)}
-   */
-  @Deprecated(since = "2.1.0")
-  public interface DispatchParmaters extends DispatchParameters {}
-
-  /**
-   * @return Should return one of the executors named params.getScanExecutors().keySet()
-   *
-   * @deprecated since 2.1.0 please implement {@link #dispatch(DispatchParameters)} instead of this.
-   *             Accumulo will only call {@link #dispatch(DispatchParameters)} directly, it will
-   *             never call this. However the default implementation of
-   *             {@link #dispatch(DispatchParameters)} calls this method.
-   */
-  @Deprecated(since = "2.1.0")
-  default String dispatch(DispatchParmaters params) {
-    throw new UnsupportedOperationException();
-  }
-
-  /**
    * The method parameters for {@link ScanDispatcher#dispatch(DispatchParameters)}. This interface
    * exists so the API can evolve and additional parameters can be passed to the method in the
    * future.
@@ -115,13 +90,5 @@
    *
    * @since 2.1.0
    */
-
-  default ScanDispatch dispatch(DispatchParameters params) {
-    String executor = dispatch((DispatchParmaters) params);
-    if (executor.equals(DefaultScanDispatch.DEFAULT_SCAN_DISPATCH.getExecutorName())) {
-      return DefaultScanDispatch.DEFAULT_SCAN_DISPATCH;
-    }
-
-    return ScanDispatch.builder().setExecutorName(executor).build();
-  }
+  ScanDispatch dispatch(DispatchParameters params);
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/scan/ScanServerSelector.java b/core/src/main/java/org/apache/accumulo/core/spi/scan/ScanServerSelector.java
index d74199a..a39450e 100644
--- a/core/src/main/java/org/apache/accumulo/core/spi/scan/ScanServerSelector.java
+++ b/core/src/main/java/org/apache/accumulo/core/spi/scan/ScanServerSelector.java
@@ -27,6 +27,7 @@
 
 import org.apache.accumulo.core.client.ScannerBase;
 import org.apache.accumulo.core.data.TabletId;
+import org.apache.accumulo.core.lock.ServiceLockData;
 import org.apache.accumulo.core.spi.common.ServiceEnvironment;
 
 import com.google.common.base.Preconditions;
@@ -46,7 +47,7 @@
   /**
    * The scan server group name that will be used when one is not specified.
    */
-  String DEFAULT_SCAN_SERVER_GROUP_NAME = "default";
+  String DEFAULT_SCAN_SERVER_GROUP_NAME = ServiceLockData.ServiceDescriptor.DEFAULT_GROUP_NAME;
 
   /**
    * This method is called once after a {@link ScanServerSelector} is instantiated.
diff --git a/core/src/main/java/org/apache/accumulo/core/summary/Gatherer.java b/core/src/main/java/org/apache/accumulo/core/summary/Gatherer.java
index 3cb22ea..e9ee45d 100644
--- a/core/src/main/java/org/apache/accumulo/core/summary/Gatherer.java
+++ b/core/src/main/java/org/apache/accumulo/core/summary/Gatherer.java
@@ -51,6 +51,7 @@
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
 import org.apache.accumulo.core.clientImpl.ClientContext;
+import org.apache.accumulo.core.clientImpl.thrift.TInfo;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.data.Key;
@@ -60,20 +61,18 @@
 import org.apache.accumulo.core.dataImpl.thrift.TRowRange;
 import org.apache.accumulo.core.dataImpl.thrift.TSummaries;
 import org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest;
-import org.apache.accumulo.core.metadata.TabletFile;
+import org.apache.accumulo.core.metadata.StoredTabletFile;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata;
 import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
 import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
 import org.apache.accumulo.core.spi.cache.BlockCache;
 import org.apache.accumulo.core.spi.crypto.CryptoService;
-import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Client;
+import org.apache.accumulo.core.tabletserver.thrift.TabletServerClientService.Client;
 import org.apache.accumulo.core.trace.TraceUtil;
-import org.apache.accumulo.core.trace.thrift.TInfo;
 import org.apache.accumulo.core.util.ByteBufferUtil;
 import org.apache.accumulo.core.util.CancelFlagFuture;
 import org.apache.accumulo.core.util.CompletableFutureUtil;
-import org.apache.accumulo.core.util.HostAndPort;
 import org.apache.accumulo.core.util.TextUtil;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -84,9 +83,10 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.github.benmanes.caffeine.cache.Cache;
 import com.google.common.base.Preconditions;
-import com.google.common.cache.Cache;
 import com.google.common.hash.Hashing;
+import com.google.common.net.HostAndPort;
 
 /**
  * This class implements using multiple tservers to gather summaries.
@@ -163,31 +163,31 @@
    * @return A map of the form : {@code map<tserver location, map<path, list<range>>} . The ranges
    *         associated with a file represent the tablets that use the file.
    */
-  private Map<String,Map<TabletFile,List<TRowRange>>>
-      getFilesGroupedByLocation(Predicate<TabletFile> fileSelector) {
-
-    Iterable<TabletMetadata> tmi = TabletsMetadata.builder(ctx).forTable(tableId)
-        .overlapping(startRow, endRow).fetch(FILES, LOCATION, LAST, PREV_ROW).build();
+  private Map<String,Map<StoredTabletFile,List<TRowRange>>>
+      getFilesGroupedByLocation(Predicate<StoredTabletFile> fileSelector) {
 
     // get a subset of files
-    Map<TabletFile,List<TabletMetadata>> files = new HashMap<>();
+    Map<StoredTabletFile,List<TabletMetadata>> files = new HashMap<>();
 
-    for (TabletMetadata tm : tmi) {
-      for (TabletFile file : tm.getFiles()) {
-        if (fileSelector.test(file)) {
-          // TODO push this filtering to server side and possibly use batch scanner
-          files.computeIfAbsent(file, s -> new ArrayList<>()).add(tm);
+    try (TabletsMetadata tmi = TabletsMetadata.builder(ctx).forTable(tableId)
+        .overlapping(startRow, endRow).fetch(FILES, LOCATION, LAST, PREV_ROW).build()) {
+      for (TabletMetadata tm : tmi) {
+        for (StoredTabletFile file : tm.getFiles()) {
+          if (fileSelector.test(file)) {
+            // TODO push this filtering to server side and possibly use batch scanner
+            files.computeIfAbsent(file, s -> new ArrayList<>()).add(tm);
+          }
         }
       }
     }
 
     // group by location, then file
 
-    Map<String,Map<TabletFile,List<TRowRange>>> locations = new HashMap<>();
+    Map<String,Map<StoredTabletFile,List<TRowRange>>> locations = new HashMap<>();
 
     List<String> tservers = null;
 
-    for (Entry<TabletFile,List<TabletMetadata>> entry : files.entrySet()) {
+    for (Entry<StoredTabletFile,List<TabletMetadata>> entry : files.entrySet()) {
 
       String location = entry.getValue().stream().filter(tm -> tm.getLocation() != null) // filter
                                                                                          // tablets
@@ -209,9 +209,8 @@
 
         // When no location, the approach below will consistently choose the same tserver for the
         // same file (as long as the set of tservers is stable).
-        int idx = Math
-            .abs(Hashing.murmur3_32_fixed().hashString(entry.getKey().getPathStr(), UTF_8).asInt())
-            % tservers.size();
+        int idx = Math.abs(Hashing.murmur3_32_fixed()
+            .hashString(entry.getKey().getNormalizedPathStr(), UTF_8).asInt()) % tservers.size();
         location = tservers.get(idx);
       }
 
@@ -261,7 +260,7 @@
 
   private static class ProcessedFiles {
     final SummaryCollection summaries;
-    final Set<TabletFile> failedFiles;
+    final Set<StoredTabletFile> failedFiles;
 
     public ProcessedFiles() {
       this.summaries = new SummaryCollection();
@@ -286,12 +285,12 @@
   private class FilesProcessor implements Supplier<ProcessedFiles> {
 
     HostAndPort location;
-    Map<TabletFile,List<TRowRange>> allFiles;
+    Map<StoredTabletFile,List<TRowRange>> allFiles;
     private TInfo tinfo;
     private AtomicBoolean cancelFlag;
 
     public FilesProcessor(TInfo tinfo, HostAndPort location,
-        Map<TabletFile,List<TRowRange>> allFiles, AtomicBoolean cancelFlag) {
+        Map<StoredTabletFile,List<TRowRange>> allFiles, AtomicBoolean cancelFlag) {
       this.location = location;
       this.allFiles = allFiles;
       this.tinfo = tinfo;
@@ -306,7 +305,7 @@
       try {
         client = ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, location, ctx);
         // partition files into smaller chunks so that not too many are sent to a tserver at once
-        for (Map<TabletFile,List<TRowRange>> files : partition(allFiles, 500)) {
+        for (Map<StoredTabletFile,List<TRowRange>> files : partition(allFiles, 500)) {
           if (!pfiles.failedFiles.isEmpty()) {
             // there was a previous failure on this tserver, so just fail the rest of the files
             pfiles.failedFiles.addAll(files.keySet());
@@ -315,8 +314,8 @@
 
           try {
             TSummaries tSums = client.startGetSummariesFromFiles(tinfo, ctx.rpcCreds(),
-                getRequest(), files.entrySet().stream().collect(
-                    Collectors.toMap(entry -> entry.getKey().getPathStr(), Entry::getValue)));
+                getRequest(), files.entrySet().stream().collect(Collectors
+                    .toMap(entry -> entry.getKey().getNormalizedPathStr(), Entry::getValue)));
             while (!tSums.finished && !cancelFlag.get()) {
               tSums = client.contiuneGetSummaries(tinfo, tSums.sessionId);
             }
@@ -326,7 +325,7 @@
             pfiles.failedFiles.addAll(files.keySet());
             continue;
           } catch (TException e) {
-            throw new RuntimeException(e);
+            throw new IllegalStateException(e);
           }
 
         }
@@ -337,7 +336,7 @@
       }
 
       if (cancelFlag.get()) {
-        throw new RuntimeException("Operation canceled");
+        throw new IllegalStateException("Operation canceled");
       }
 
       return pfiles;
@@ -350,13 +349,13 @@
 
     PartitionFuture(TInfo tinfo, ExecutorService execSrv, int modulus, int remainder) {
       Function<ProcessedFiles,CompletableFuture<ProcessedFiles>> go = previousWork -> {
-        Predicate<TabletFile> fileSelector = file -> Math
-            .abs(Hashing.murmur3_32_fixed().hashString(file.getPathStr(), UTF_8).asInt()) % modulus
-            == remainder;
+        Predicate<StoredTabletFile> fileSelector = file -> Math
+            .abs(Hashing.murmur3_32_fixed().hashString(file.getNormalizedPathStr(), UTF_8).asInt())
+            % modulus == remainder;
         if (previousWork != null) {
           fileSelector = fileSelector.and(previousWork.failedFiles::contains);
         }
-        Map<String,Map<TabletFile,List<TRowRange>>> filesGBL;
+        Map<String,Map<StoredTabletFile,List<TRowRange>>> filesGBL;
         filesGBL = getFilesGroupedByLocation(fileSelector);
 
         List<CompletableFuture<ProcessedFiles>> futures = new ArrayList<>();
@@ -365,9 +364,9 @@
               .completedFuture(new ProcessedFiles(previousWork.summaries, factory)));
         }
 
-        for (Entry<String,Map<TabletFile,List<TRowRange>>> entry : filesGBL.entrySet()) {
+        for (Entry<String,Map<StoredTabletFile,List<TRowRange>>> entry : filesGBL.entrySet()) {
           HostAndPort location = HostAndPort.fromString(entry.getKey());
-          Map<TabletFile,List<TRowRange>> allFiles = entry.getValue();
+          Map<StoredTabletFile,List<TRowRange>> allFiles = entry.getValue();
 
           futures.add(CompletableFuture
               .supplyAsync(new FilesProcessor(tinfo, location, allFiles, cancelFlag), execSrv));
@@ -448,8 +447,10 @@
 
   private int countFiles() {
     // TODO use a batch scanner + iterator to parallelize counting files
-    return TabletsMetadata.builder(ctx).forTable(tableId).overlapping(startRow, endRow)
-        .fetch(FILES, PREV_ROW).build().stream().mapToInt(tm -> tm.getFiles().size()).sum();
+    try (TabletsMetadata tabletsMetadata = TabletsMetadata.builder(ctx).forTable(tableId)
+        .overlapping(startRow, endRow).fetch(FILES, PREV_ROW).build()) {
+      return tabletsMetadata.stream().mapToInt(tm -> tm.getFiles().size()).sum();
+    }
   }
 
   private class GatherRequest implements Supplier<SummaryCollection> {
@@ -481,11 +482,11 @@
           return tsr;
         });
       } catch (AccumuloException | AccumuloSecurityException e) {
-        throw new RuntimeException(e);
+        throw new IllegalStateException(e);
       }
 
       if (cancelFlag.get()) {
-        throw new RuntimeException("Operation canceled");
+        throw new IllegalStateException("Operation canceled");
       }
 
       return new SummaryCollection(tSums);
diff --git a/core/src/main/java/org/apache/accumulo/core/summary/SummarizerFactory.java b/core/src/main/java/org/apache/accumulo/core/summary/SummarizerFactory.java
index 48b123b..5b15a90 100644
--- a/core/src/main/java/org/apache/accumulo/core/summary/SummarizerFactory.java
+++ b/core/src/main/java/org/apache/accumulo/core/summary/SummarizerFactory.java
@@ -19,6 +19,7 @@
 package org.apache.accumulo.core.summary;
 
 import java.io.IOException;
+import java.io.UncheckedIOException;
 
 import org.apache.accumulo.core.classloader.ClassLoaderUtil;
 import org.apache.accumulo.core.client.summary.Summarizer;
@@ -55,8 +56,10 @@
   public Summarizer getSummarizer(SummarizerConfiguration conf) {
     try {
       return newSummarizer(conf.getClassName());
-    } catch (ReflectiveOperationException | IOException e) {
-      throw new RuntimeException(e);
+    } catch (ReflectiveOperationException e) {
+      throw new IllegalStateException(e);
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
     }
   }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/summary/SummaryReader.java b/core/src/main/java/org/apache/accumulo/core/summary/SummaryReader.java
index b4f96de..ed69454 100644
--- a/core/src/main/java/org/apache/accumulo/core/summary/SummaryReader.java
+++ b/core/src/main/java/org/apache/accumulo/core/summary/SummaryReader.java
@@ -46,7 +46,7 @@
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.WritableUtils;
 
-import com.google.common.cache.Cache;
+import com.github.benmanes.caffeine.cache.Cache;
 
 public class SummaryReader {
 
diff --git a/core/src/main/java/org/apache/accumulo/core/tabletserver/log/LogEntry.java b/core/src/main/java/org/apache/accumulo/core/tabletserver/log/LogEntry.java
index efcd26c..6bd5ba5 100644
--- a/core/src/main/java/org/apache/accumulo/core/tabletserver/log/LogEntry.java
+++ b/core/src/main/java/org/apache/accumulo/core/tabletserver/log/LogEntry.java
@@ -18,97 +18,171 @@
  */
 package org.apache.accumulo.core.tabletserver.log;
 
-import java.io.IOException;
-import java.util.Arrays;
 import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.UUID;
 
 import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LogColumnFamily;
-import org.apache.hadoop.io.DataInputBuffer;
-import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.Text;
+import org.checkerframework.checker.nullness.qual.NonNull;
 
-public class LogEntry {
-  private final KeyExtent extent;
-  public final long timestamp;
-  public final String filename;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.net.HostAndPort;
 
-  public LogEntry(KeyExtent extent, long timestamp, String filename) {
-    // note the prevEndRow in the extent does not matter, and is not used by LogEntry
-    this.extent = extent;
-    this.timestamp = timestamp;
-    this.filename = filename;
+public final class LogEntry {
+
+  private final String path;
+  private final HostAndPort tserver;
+  private final UUID uniqueId;
+  private final Text columnQualifier;
+
+  private LogEntry(String path, HostAndPort tserver, UUID uniqueId, Text columnQualifier) {
+    this.path = path;
+    this.tserver = tserver;
+    this.uniqueId = uniqueId;
+    this.columnQualifier = columnQualifier;
   }
 
-  // make copy, but with a different filename
-  public LogEntry switchFile(String filename) {
-    return new LogEntry(extent, timestamp, filename);
+  /**
+   * Creates a new LogEntry object after validating the expected format of the path. We expect the
+   * path to contain a tserver (host+port) followed by a UUID as the file name as the last two
+   * components.<br>
+   * For example, file:///some/dir/path/localhost+1234/927ba659-d109-4bce-b0a5-bcbbcb9942a2 is a
+   * valid path.
+   *
+   * @param path path to validate
+   * @return an object representation of this log entry
+   * @throws IllegalArgumentException if the path is invalid
+   */
+  public static LogEntry fromPath(String path) {
+    return validatedLogEntry(path, null);
+  }
+
+  private static LogEntry validatedLogEntry(String path, Text columnQualifier) {
+    String[] parts = path.split("/");
+
+    if (parts.length < 2) {
+      throw new IllegalArgumentException(
+          "Invalid path format. The path should end with tserver/UUID.");
+    }
+
+    String tserverPart = parts[parts.length - 2];
+    String uuidPart = parts[parts.length - 1];
+
+    String badTServerMsg =
+        "Invalid tserver in path. Expected: host+port. Found '" + tserverPart + "'";
+    if (tserverPart.contains(":") || !tserverPart.contains("+")) {
+      throw new IllegalArgumentException(badTServerMsg);
+    }
+    HostAndPort tserver;
+    try {
+      tserver = HostAndPort.fromString(tserverPart.replace("+", ":"));
+    } catch (IllegalArgumentException e) {
+      throw new IllegalArgumentException(badTServerMsg);
+    }
+
+    String badUuidMsg = "Expected valid UUID. Found '" + uuidPart + "'";
+    UUID uuid;
+    try {
+      uuid = UUID.fromString(uuidPart);
+    } catch (IllegalArgumentException e) {
+      throw new IllegalArgumentException(badUuidMsg);
+    }
+    if (!uuid.toString().equals(uuidPart)) {
+      throw new IllegalArgumentException(badUuidMsg);
+    }
+
+    return new LogEntry(path, tserver, uuid, columnQualifier);
+  }
+
+  /**
+   * Construct a new LogEntry object after deserializing it from a metadata entry.
+   *
+   * @param entry the metadata entry
+   * @return a new LogEntry object constructed from the path stored in the column qualifier
+   * @throws IllegalArgumentException if the path stored in the metadata entry is invalid or if the
+   *         serialized format of the entry is unrecognized
+   */
+  public static LogEntry fromMetaWalEntry(Entry<Key,Value> entry) {
+    Text fam = entry.getKey().getColumnFamily();
+    Preconditions.checkArgument(LogColumnFamily.NAME.equals(fam),
+        "The provided metadata entry's column family is %s instead of %s", fam,
+        LogColumnFamily.NAME);
+    Text qualifier = entry.getKey().getColumnQualifier();
+    String[] parts = qualifier.toString().split("/", 2);
+    Preconditions.checkArgument(parts.length == 2, "Malformed write-ahead log %s", qualifier);
+    return validatedLogEntry(parts[1], qualifier);
+  }
+
+  @NonNull
+  @VisibleForTesting
+  HostAndPort getTServer() {
+    return tserver;
+  }
+
+  @NonNull
+  public String getPath() {
+    return path;
+  }
+
+  @NonNull
+  public UUID getUniqueID() {
+    return uniqueId;
   }
 
   @Override
   public String toString() {
-    return extent.toMetaRow() + " " + filename;
+    return path;
   }
 
-  // unused; kept only for reference with corresponding fromBytes method
-  @Deprecated(since = "2.1.0", forRemoval = true)
-  public byte[] toBytes() throws IOException {
-    DataOutputBuffer out = new DataOutputBuffer();
-    extent.writeTo(out);
-    out.writeLong(timestamp);
-    // this next string used to store server, but this is no longer used
-    out.writeUTF("-");
-    out.writeUTF(filename);
-    return Arrays.copyOf(out.getData(), out.getLength());
+  @Override
+  public boolean equals(Object other) {
+    if (this == other) {
+      return true;
+    }
+    if (other instanceof LogEntry) {
+      return path.equals(((LogEntry) other).path);
+    }
+    return false;
   }
 
-  // kept only for upgrade code to upgrade WAL entries for the root table
-  @Deprecated(since = "2.1.0", forRemoval = true)
-  public static LogEntry fromBytes(byte[] bytes) throws IOException {
-    DataInputBuffer inp = new DataInputBuffer();
-    inp.reset(bytes, bytes.length);
-    KeyExtent extent = KeyExtent.readFrom(inp);
-    long timestamp = inp.readLong();
-    // this next string used to store the server, but this is no longer used
-    inp.readUTF();
-    String filename = inp.readUTF();
-    return new LogEntry(extent, timestamp, filename);
+  @Override
+  public int hashCode() {
+    return Objects.hash(path);
   }
 
-  public static LogEntry fromMetaWalEntry(Entry<Key,Value> entry) {
-    final Key key = entry.getKey();
-    final Value value = entry.getValue();
-    KeyExtent extent = KeyExtent.fromMetaRow(key.getRow());
-    // qualifier.split("/")[0] used to store the server, but this is no longer used, and the
-    // qualifier can be ignored
-    // the following line handles old-style log entry values that specify log sets
-    String[] parts = value.toString().split("\\|")[0].split(";");
-    String filename = parts[parts.length - 1];
-    long timestamp = key.getTimestamp();
-    return new LogEntry(extent, timestamp, filename);
+  /**
+   * Get the Text that should be used as the column qualifier to store this as a metadata entry.
+   */
+  @VisibleForTesting
+  Text getColumnQualifier() {
+    return columnQualifier == null ? newCQ() : new Text(columnQualifier);
   }
 
-  public Text getRow() {
-    return extent.toMetaRow();
+  private Text newCQ() {
+    return new Text("-/" + getPath());
   }
 
-  public Text getColumnFamily() {
-    return LogColumnFamily.NAME;
+  /**
+   * Put a delete marker in the provided mutation for this LogEntry.
+   *
+   * @param mutation the mutation to update
+   */
+  public void deleteFromMutation(Mutation mutation) {
+    mutation.putDelete(LogColumnFamily.NAME, getColumnQualifier());
   }
 
-  public String getUniqueID() {
-    String[] parts = filename.split("/");
-    return parts[parts.length - 1];
-  }
-
-  public Text getColumnQualifier() {
-    return new Text("-/" + filename);
-  }
-
-  public Value getValue() {
-    return new Value(filename);
+  /**
+   * Put this LogEntry into the provided mutation.
+   *
+   * @param mutation the mutation to update
+   */
+  public void addToMutation(Mutation mutation) {
+    mutation.put(LogColumnFamily.NAME, newCQ(), new Value());
   }
 
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/trace/TraceUtil.java b/core/src/main/java/org/apache/accumulo/core/trace/TraceUtil.java
index 0c85bf4..78d658a 100644
--- a/core/src/main/java/org/apache/accumulo/core/trace/TraceUtil.java
+++ b/core/src/main/java/org/apache/accumulo/core/trace/TraceUtil.java
@@ -25,9 +25,9 @@
 import java.util.concurrent.Callable;
 
 import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.clientImpl.thrift.TInfo;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.trace.thrift.TInfo;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -131,8 +131,8 @@
   }
 
   /**
-   * Obtain {@link org.apache.accumulo.core.trace.thrift.TInfo} for the current context. This is
-   * used to send the current trace information to a remote process
+   * Obtain {@link org.apache.accumulo.core.clientImpl.thrift.TInfo} for the current context. This
+   * is used to send the current trace information to a remote process
    */
   public static TInfo traceInfo() {
     TInfo tinfo = new TInfo();
diff --git a/core/src/main/java/org/apache/accumulo/core/util/AddressUtil.java b/core/src/main/java/org/apache/accumulo/core/util/AddressUtil.java
index 5fb0d40..b6bb6f9 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/AddressUtil.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/AddressUtil.java
@@ -25,15 +25,16 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.net.HostAndPort;
+
 public class AddressUtil {
 
   private static final Logger log = LoggerFactory.getLogger(AddressUtil.class);
 
-  public static HostAndPort parseAddress(String address, boolean ignoreMissingPort)
-      throws NumberFormatException {
-    address = address.replace('+', ':');
-    HostAndPort hap = HostAndPort.fromString(address);
-    if (!ignoreMissingPort && !hap.hasPort()) {
+  public static HostAndPort parseAddress(final String address) throws NumberFormatException {
+    String normalized = normalizePortSeparator(address);
+    HostAndPort hap = HostAndPort.fromString(normalized);
+    if (!hap.hasPort()) {
       throw new IllegalArgumentException(
           "Address was expected to contain port. address=" + address);
     }
@@ -41,8 +42,13 @@
     return hap;
   }
 
-  public static HostAndPort parseAddress(String address, int defaultPort) {
-    return parseAddress(address, true).withDefaultPort(defaultPort);
+  public static HostAndPort parseAddress(final String address, final int defaultPort) {
+    String normalized = normalizePortSeparator(address);
+    return HostAndPort.fromString(normalized).withDefaultPort(defaultPort);
+  }
+
+  private static String normalizePortSeparator(final String address) {
+    return address.replace('+', ':');
   }
 
   /**
diff --git a/core/src/main/java/org/apache/accumulo/core/util/BadArgumentException.java b/core/src/main/java/org/apache/accumulo/core/util/BadArgumentException.java
index ac9cafe..374c4f8 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/BadArgumentException.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/BadArgumentException.java
@@ -20,10 +20,17 @@
 
 import java.util.regex.PatternSyntaxException;
 
+import org.apache.accumulo.access.InvalidAccessExpressionException;
+
 public final class BadArgumentException extends PatternSyntaxException {
   private static final long serialVersionUID = 1L;
 
   public BadArgumentException(String desc, String badarg, int index) {
     super(desc, badarg, index);
   }
+
+  public BadArgumentException(InvalidAccessExpressionException e) {
+    super(e.getDescription(), e.getPattern(), e.getIndex());
+    super.initCause(e);
+  }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/util/CleanUp.java b/core/src/main/java/org/apache/accumulo/core/util/CleanUp.java
deleted file mode 100644
index 0324a3b..0000000
--- a/core/src/main/java/org/apache/accumulo/core/util/CleanUp.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.util;
-
-import java.util.Set;
-
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.clientImpl.ConnectorImpl;
-import org.apache.accumulo.core.singletons.SingletonManager;
-import org.apache.accumulo.core.singletons.SingletonManager.Mode;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Connector uses static resource that create threads and use memory. The only way to clean up these
- * static resource related to Connectors created using ZookeeperInstance is to use this class.
- *
- * <p>
- * This class is not needed when only using {@link AccumuloClient}. The new AccumuloClient API that
- * replaces Connector is closable. For code that only uses AccumuloClient, when all AccumuloClients
- * are closed resources are cleaned up. Connectors that are derived from an AccumuloClient do not
- * necessitate the use of this code.
- *
- * @deprecated since 2.0.0 Use only {@link AccumuloClient} instead. Also, make sure you close the
- *             AccumuloClient instances.
- */
-@Deprecated(since = "2.0.0")
-public class CleanUp {
-
-  private static final Logger log = LoggerFactory.getLogger(CleanUp.class);
-
-  /**
-   * kills all threads created by internal Accumulo singleton resources. After this method is
-   * called, no Connector will work in the current classloader.
-   *
-   * @param conn If available, Connector object to close resources on. Will accept null otherwise.
-   */
-  public static void shutdownNow(Connector conn) {
-    SingletonManager.setMode(Mode.CLIENT);
-    waitForZooKeeperClientThreads();
-    if (conn != null) {
-      ConnectorImpl connImpl = (ConnectorImpl) conn;
-      connImpl.getAccumuloClient().close();
-    }
-  }
-
-  /**
-   * As documented in https://issues.apache.org/jira/browse/ZOOKEEPER-1816, ZooKeeper.close() is a
-   * non-blocking call. This method will wait on the ZooKeeper internal threads to exit.
-   */
-  private static void waitForZooKeeperClientThreads() {
-    Set<Thread> threadSet = Thread.getAllStackTraces().keySet();
-    for (Thread thread : threadSet) {
-      // find ZooKeeper threads that were created in the same ClassLoader as the current thread.
-      if (thread.getClass().getName().startsWith("org.apache.zookeeper.ClientCnxn") && thread
-          .getContextClassLoader().equals(Thread.currentThread().getContextClassLoader())) {
-
-        // wait for the thread the die
-        while (thread.isAlive()) {
-          try {
-            Thread.sleep(100);
-          } catch (InterruptedException e) {
-            log.error("{}", e.getMessage(), e);
-          }
-        }
-      }
-    }
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/util/CreateToken.java b/core/src/main/java/org/apache/accumulo/core/util/CreateToken.java
index a545c0d..cccd00f 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/CreateToken.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/CreateToken.java
@@ -86,34 +86,35 @@
       pass = opts.securePassword;
     }
 
-    try {
-      String principal = opts.principal;
-      if (principal == null) {
-        principal = getConsoleReader().readLine("Username (aka principal): ");
-      }
-
-      AuthenticationToken token = Class.forName(opts.tokenClassName)
-          .asSubclass(AuthenticationToken.class).getDeclaredConstructor().newInstance();
-      Properties props = new Properties();
-      for (TokenProperty tp : token.getProperties()) {
-        String input;
-        if (pass != null && tp.getKey().equals("password")) {
-          input = pass;
-        } else {
-          if (tp.getMask()) {
-            input = getConsoleReader().readLine(tp.getDescription() + ": ", '*');
-          } else {
-            input = getConsoleReader().readLine(tp.getDescription() + ": ");
-          }
-        }
-        props.put(tp.getKey(), input);
-        token.init(props);
-      }
-      System.out.println("auth.type = " + opts.tokenClassName);
-      System.out.println("auth.principal = " + principal);
-      System.out.println("auth.token = " + ClientProperty.encodeToken(token));
-    } catch (ReflectiveOperationException e) {
-      throw new RuntimeException(e);
+    String principal = opts.principal;
+    if (principal == null) {
+      principal = getConsoleReader().readLine("Username (aka principal): ");
     }
+
+    AuthenticationToken token;
+    try {
+      token = Class.forName(opts.tokenClassName).asSubclass(AuthenticationToken.class)
+          .getDeclaredConstructor().newInstance();
+    } catch (ReflectiveOperationException e) {
+      throw new IllegalStateException(e);
+    }
+    Properties props = new Properties();
+    for (TokenProperty tp : token.getProperties()) {
+      String input;
+      if (pass != null && tp.getKey().equals("password")) {
+        input = pass;
+      } else {
+        if (tp.getMask()) {
+          input = getConsoleReader().readLine(tp.getDescription() + ": ", '*');
+        } else {
+          input = getConsoleReader().readLine(tp.getDescription() + ": ");
+        }
+      }
+      props.put(tp.getKey(), input);
+      token.init(props);
+    }
+    System.out.println("auth.type = " + opts.tokenClassName);
+    System.out.println("auth.principal = " + principal);
+    System.out.println("auth.token = " + ClientProperty.encodeToken(token));
   }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/util/FastFormat.java b/core/src/main/java/org/apache/accumulo/core/util/FastFormat.java
index f762110..da693b0 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/FastFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/FastFormat.java
@@ -31,8 +31,8 @@
     String strNum = Long.toString(num, radix);
     byte[] ret = new byte[Math.max(strNum.length(), width) + prefix.length];
     if (toZeroPaddedString(ret, 0, strNum, width, prefix) != ret.length) {
-      throw new RuntimeException(" Did not format to expected width " + num + " " + width + " "
-          + radix + " " + new String(prefix, UTF_8));
+      throw new IllegalArgumentException(" Did not format to expected width " + num + " " + width
+          + " " + radix + " " + new String(prefix, UTF_8));
     }
     return ret;
   }
diff --git a/core/src/main/java/org/apache/accumulo/core/util/Halt.java b/core/src/main/java/org/apache/accumulo/core/util/Halt.java
index ec822b4..da4939a 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/Halt.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/Halt.java
@@ -18,8 +18,8 @@
  */
 package org.apache.accumulo.core.util;
 
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
 
 import org.apache.accumulo.core.util.threads.Threads;
 import org.slf4j.Logger;
diff --git a/core/src/main/java/org/apache/accumulo/core/util/HostAndPort.java b/core/src/main/java/org/apache/accumulo/core/util/HostAndPort.java
deleted file mode 100644
index 08c5569..0000000
--- a/core/src/main/java/org/apache/accumulo/core/util/HostAndPort.java
+++ /dev/null
@@ -1,289 +0,0 @@
-/*
- * Copyright (C) 2011 The Guava Authors
- *
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- *     https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.core.util;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkState;
-
-import java.io.Serializable;
-import java.util.Comparator;
-import java.util.Objects;
-
-/**
- * This class was copied from Guava release 23.0 to replace the older Guava 14 version that had been
- * used in Accumulo. It was annotated as Beta by Google, therefore unstable to use in a core
- * Accumulo library. We learned this the hard way when Guava version 20 deprecated the getHostText
- * method and then removed the method all together in version 22. See ACCUMULO-4702
- *
- * Unused methods and annotations were removed to reduce maintenance costs.
- *
- * Javadoc from Guava 23.0 release: An immutable representation of a host and port.
- *
- * <p>
- * Example usage:
- *
- * <pre>
- * HostAndPort hp =
- *     HostAndPort.fromString(&quot;[2001:db8::1]&quot;).withDefaultPort(80).requireBracketsForIPv6();
- * hp.getHost(); // returns &quot;2001:db8::1&quot;
- * hp.getPort(); // returns 80
- * hp.toString(); // returns &quot;[2001:db8::1]:80&quot;
- * </pre>
- *
- * <p>
- * Here are some examples of recognized formats:
- * <ul>
- * <li>example.com
- * <li>example.com:80
- * <li>192.0.2.1
- * <li>192.0.2.1:80
- * <li>[2001:db8::1] - {@link #getHost()} omits brackets
- * <li>[2001:db8::1]:80 - {@link #getHost()} omits brackets
- * <li>2001:db8::1 - Use requireBracketsForIPv6() to prohibit this
- * </ul>
- *
- * <p>
- * Note that this is not an exhaustive list, because these methods are only concerned with brackets,
- * colons, and port numbers. Full validation of the host field (if desired) is the caller's
- * responsibility.
- */
-public final class HostAndPort implements Serializable, Comparable<HostAndPort> {
-  /** Magic value indicating the absence of a port number. */
-  private static final int NO_PORT = -1;
-
-  /** Hostname, IPv4/IPv6 literal, or unvalidated nonsense. */
-  private final String host;
-
-  /** Validated port number in the range [0..65535], or NO_PORT */
-  private final int port;
-
-  /** True if the parsed host has colons, but no surrounding brackets. */
-  private final boolean hasBracketlessColons;
-
-  private HostAndPort(String host, int port, boolean hasBracketlessColons) {
-    this.host = host;
-    this.port = port;
-    this.hasBracketlessColons = hasBracketlessColons;
-  }
-
-  private static final Comparator<HostAndPort> COMPARATOR = Comparator.nullsFirst(
-      Comparator.comparing(HostAndPort::getHost).thenComparingInt(h -> h.getPortOrDefault(0)));
-
-  /**
-   * Returns the portion of this {@code HostAndPort} instance that should represent the hostname or
-   * IPv4/IPv6 literal.
-   *
-   * <p>
-   * A successful parse does not imply any degree of sanity in this field. For additional
-   * validation, see the HostSpecifier class.
-   *
-   * @since 20.0 (since 10.0 as {@code getHostText})
-   */
-  public String getHost() {
-    return host;
-  }
-
-  /** Return true if this instance has a defined port. */
-  public boolean hasPort() {
-    return port >= 0;
-  }
-
-  /**
-   * Get the current port number, failing if no port is defined.
-   *
-   * @return a validated port number, in the range [0..65535]
-   * @throws IllegalStateException if no port is defined. You can use {@link #withDefaultPort(int)}
-   *         to prevent this from occurring.
-   */
-  public int getPort() {
-    checkState(hasPort(), "the address does not include a port");
-    return port;
-  }
-
-  /**
-   * Build a HostAndPort instance from separate host and port values.
-   *
-   * <p>
-   * Note: Non-bracketed IPv6 literals are allowed. Use #requireBracketsForIPv6() to prohibit these.
-   *
-   * @param host the host string to parse. Must not contain a port number.
-   * @param port a port number from [0..65535]
-   * @return if parsing was successful, a populated HostAndPort object.
-   * @throws IllegalArgumentException if {@code host} contains a port number, or {@code port} is out
-   *         of range.
-   */
-  public static HostAndPort fromParts(String host, int port) {
-    checkArgument(isValidPort(port), "Port out of range: %s", port);
-    HostAndPort parsedHost = fromString(host);
-    checkArgument(!parsedHost.hasPort(), "Host has a port: %s", host);
-    return new HostAndPort(parsedHost.host, port, parsedHost.hasBracketlessColons);
-  }
-
-  /**
-   * Split a freeform string into a host and port, without strict validation.
-   *
-   * Note that the host-only formats will leave the port field undefined. You can use
-   * {@link #withDefaultPort(int)} to patch in a default value.
-   *
-   * @param hostPortString the input string to parse.
-   * @return if parsing was successful, a populated HostAndPort object.
-   * @throws IllegalArgumentException if nothing meaningful could be parsed.
-   */
-  public static HostAndPort fromString(String hostPortString) {
-    Objects.requireNonNull(hostPortString, "hostPortString variable was null!");
-    String host;
-    String portString = null;
-    boolean hasBracketlessColons = false;
-
-    if (hostPortString.startsWith("[")) {
-      String[] hostAndPort = getHostAndPortFromBracketedHost(hostPortString);
-      host = hostAndPort[0];
-      portString = hostAndPort[1];
-    } else {
-      int colonPos = hostPortString.indexOf(':');
-      if (colonPos >= 0 && hostPortString.indexOf(':', colonPos + 1) == -1) {
-        // Exactly 1 colon. Split into host:port.
-        host = hostPortString.substring(0, colonPos);
-        portString = hostPortString.substring(colonPos + 1);
-      } else {
-        // 0 or 2+ colons. Bare hostname or IPv6 literal.
-        host = hostPortString;
-        hasBracketlessColons = (colonPos >= 0);
-      }
-    }
-
-    int port = NO_PORT;
-    if (portString != null && !portString.trim().isEmpty()) {
-      // Try to parse the whole port string as a number.
-      // JDK7 accepts leading plus signs. We don't want to.
-      checkArgument(!portString.startsWith("+"), "Unparseable port number: %s", hostPortString);
-      try {
-        port = Integer.parseInt(portString);
-      } catch (NumberFormatException e) {
-        throw new IllegalArgumentException("Unparseable port number: " + hostPortString);
-      }
-      checkArgument(isValidPort(port), "Port number out of range: %s", hostPortString);
-    }
-
-    return new HostAndPort(host, port, hasBracketlessColons);
-  }
-
-  /**
-   * Parses a bracketed host-port string, throwing IllegalArgumentException if parsing fails.
-   *
-   * @param hostPortString the full bracketed host-port specification. Post might not be specified.
-   * @return an array with 2 strings: host and port, in that order.
-   * @throws IllegalArgumentException if parsing the bracketed host-port string fails.
-   */
-  private static String[] getHostAndPortFromBracketedHost(String hostPortString) {
-    int colonIndex = 0;
-    int closeBracketIndex = 0;
-    checkArgument(hostPortString.charAt(0) == '[',
-        "Bracketed host-port string must start with a bracket: %s", hostPortString);
-    colonIndex = hostPortString.indexOf(':');
-    closeBracketIndex = hostPortString.lastIndexOf(']');
-    checkArgument(colonIndex > -1 && closeBracketIndex > colonIndex,
-        "Invalid bracketed host/port: %s", hostPortString);
-
-    String host = hostPortString.substring(1, closeBracketIndex);
-    if (closeBracketIndex + 1 == hostPortString.length()) {
-      return new String[] {host, ""};
-    } else {
-      checkArgument(hostPortString.charAt(closeBracketIndex + 1) == ':',
-          "Only a colon may follow a close bracket: %s", hostPortString);
-      for (int i = closeBracketIndex + 2; i < hostPortString.length(); ++i) {
-        checkArgument(Character.isDigit(hostPortString.charAt(i)), "Port must be numeric: %s",
-            hostPortString);
-      }
-      return new String[] {host, hostPortString.substring(closeBracketIndex + 2)};
-    }
-  }
-
-  /**
-   * Provide a default port if the parsed string contained only a host.
-   *
-   * You can chain this after {@link #fromString(String)} to include a port in case the port was
-   * omitted from the input string. If a port was already provided, then this method is a no-op.
-   *
-   * @param defaultPort a port number, from [0..65535]
-   * @return a HostAndPort instance, guaranteed to have a defined port.
-   */
-  public HostAndPort withDefaultPort(int defaultPort) {
-    checkArgument(isValidPort(defaultPort));
-    if (hasPort() || port == defaultPort) {
-      return this;
-    }
-    return new HostAndPort(host, defaultPort, hasBracketlessColons);
-  }
-
-  @Override
-  public boolean equals(Object other) {
-    if (this == other) {
-      return true;
-    }
-    if (other instanceof HostAndPort) {
-      HostAndPort that = (HostAndPort) other;
-      return java.util.Objects.equals(this.host, that.host) && this.port == that.port
-          && this.hasBracketlessColons == that.hasBracketlessColons;
-    }
-    return false;
-  }
-
-  @Override
-  public int hashCode() {
-    return java.util.Objects.hash(host, port, hasBracketlessColons);
-  }
-
-  /** Rebuild the host:port string, including brackets if necessary. */
-  @Override
-  public String toString() {
-    // "[]:12345" requires 8 extra bytes.
-    StringBuilder builder = new StringBuilder(host.length() + 8);
-    if (host.indexOf(':') >= 0) {
-      builder.append('[').append(host).append(']');
-    } else {
-      builder.append(host);
-    }
-    if (hasPort()) {
-      builder.append(':').append(port);
-    }
-    return builder.toString();
-  }
-
-  /** Return true for valid port numbers. */
-  private static boolean isValidPort(int port) {
-    return port >= 0 && port <= 65535;
-  }
-
-  private static final long serialVersionUID = 0;
-
-  /**
-   * Returns the current port number, with a default if no port is defined.
-   */
-  public int getPortOrDefault(int defaultPort) {
-    return hasPort() ? port : defaultPort;
-  }
-
-  /**
-   * HostAndPort must implement compareTo. This method orders HostAndPort values using a String
-   * compare on the Host value with a secondary integer compare on the Port value.
-   */
-  @Override
-  public int compareTo(HostAndPort other) {
-    return COMPARATOR.compare(this, other);
-  }
-
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/clients/ReplicationServicerThriftClient.java b/core/src/main/java/org/apache/accumulo/core/util/HostAndPortComparator.java
similarity index 63%
copy from core/src/main/java/org/apache/accumulo/core/rpc/clients/ReplicationServicerThriftClient.java
copy to core/src/main/java/org/apache/accumulo/core/util/HostAndPortComparator.java
index d347979..548267f 100644
--- a/core/src/main/java/org/apache/accumulo/core/rpc/clients/ReplicationServicerThriftClient.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/HostAndPortComparator.java
@@ -16,14 +16,19 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.accumulo.core.rpc.clients;
+package org.apache.accumulo.core.util;
 
-import org.apache.accumulo.core.replication.thrift.ReplicationServicer.Client;
+import java.util.Comparator;
 
-public class ReplicationServicerThriftClient extends ThriftClientTypes<Client> {
+import com.google.common.net.HostAndPort;
 
-  ReplicationServicerThriftClient(String serviceName) {
-    super(serviceName, new Client.Factory());
+public class HostAndPortComparator implements Comparator<HostAndPort> {
+
+  private static final Comparator<HostAndPort> COMPARATOR = Comparator.nullsFirst(
+      Comparator.comparing(HostAndPort::getHost).thenComparingInt(h -> h.getPortOrDefault(0)));
+
+  @Override
+  public int compare(HostAndPort o1, HostAndPort o2) {
+    return COMPARATOR.compare(o1, o2);
   }
-
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/util/LazySingletons.java b/core/src/main/java/org/apache/accumulo/core/util/LazySingletons.java
new file mode 100644
index 0000000..aa57737
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/util/LazySingletons.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.util;
+
+import java.security.SecureRandom;
+import java.util.function.Supplier;
+
+import com.google.common.base.Suppliers;
+import com.google.gson.Gson;
+
+/**
+ * This class provides easy access to global, immutable, lazily-instantiated, and thread-safe
+ * singleton resources. These should be used with static imports.
+ */
+public class LazySingletons {
+
+  // prevent instantiating this utility class
+  private LazySingletons() {}
+
+  /**
+   * A Gson instance constructed with defaults. Construct your own if you need custom settings.
+   */
+  public static final Supplier<Gson> GSON = Suppliers.memoize(Gson::new);
+
+  /**
+   * A SecureRandom instance created with the default constructor.
+   */
+  public static final Supplier<SecureRandom> RANDOM = Suppliers.memoize(SecureRandom::new);
+
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/util/Merge.java b/core/src/main/java/org/apache/accumulo/core/util/Merge.java
index 41f5a67..4eba179 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/Merge.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/Merge.java
@@ -36,7 +36,7 @@
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
-import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.metadata.schema.DataFileValue;
 import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
 import org.apache.accumulo.core.trace.TraceUtil;
@@ -144,24 +144,48 @@
   public void mergomatic(AccumuloClient client, String table, Text start, Text end, long goalSize,
       boolean force) throws MergeException {
     try {
-      if (table.equals(MetadataTable.NAME)) {
+      if (table.equals(AccumuloTable.METADATA.tableName())) {
         throw new IllegalArgumentException("cannot merge tablets on the metadata table");
       }
       List<Size> sizes = new ArrayList<>();
       long totalSize = 0;
-      // Merge any until you get larger than the goal size, and then merge one less tablet
-      Iterator<Size> sizeIterator = getSizeIterator(client, table, start, end);
-      while (sizeIterator.hasNext()) {
-        Size next = sizeIterator.next();
-        totalSize += next.size;
-        sizes.add(next);
-        if (totalSize > goalSize) {
-          totalSize = mergeMany(client, table, sizes, goalSize, force, false);
+
+      TableId tableId;
+      ClientContext context = (ClientContext) client;
+      try {
+        tableId = context.getTableId(table);
+      } catch (Exception e) {
+        throw new MergeException(e);
+      }
+
+      try (TabletsMetadata tablets = TabletsMetadata.builder(context).scanMetadataTable()
+          .overRange(new KeyExtent(tableId, end, start).toMetaRange()).fetch(FILES, PREV_ROW)
+          .build()) {
+
+        Iterator<Size> sizeIterator = tablets.stream().map(tm -> {
+          long size = tm.getFilesMap().values().stream().mapToLong(DataFileValue::getSize).sum();
+          return new Size(tm.getExtent(), size);
+        }).iterator();
+
+        while (sizeIterator.hasNext()) {
+          Size next = sizeIterator.next();
+          totalSize += next.size;
+          sizes.add(next);
+
+          if (totalSize > goalSize) {
+            mergeMany(client, table, sizes, goalSize, force, false);
+            sizes.clear();
+            sizes.add(next);
+            totalSize = next.size;
+          }
         }
       }
+
+      // merge one less tablet
       if (sizes.size() > 1) {
         mergeMany(client, table, sizes, goalSize, force, true);
       }
+
     } catch (Exception ex) {
       throw new MergeException(ex);
     }
@@ -239,26 +263,4 @@
     }
   }
 
-  protected Iterator<Size> getSizeIterator(AccumuloClient client, String tablename, Text start,
-      Text end) throws MergeException {
-    // open up metadata, walk through the tablets.
-
-    TableId tableId;
-    TabletsMetadata tablets;
-    try {
-      ClientContext context = (ClientContext) client;
-      tableId = context.getTableId(tablename);
-      tablets = TabletsMetadata.builder(context).scanMetadataTable()
-          .overRange(new KeyExtent(tableId, end, start).toMetaRange()).fetch(FILES, PREV_ROW)
-          .build();
-    } catch (Exception e) {
-      throw new MergeException(e);
-    }
-
-    return tablets.stream().map(tm -> {
-      long size = tm.getFilesMap().values().stream().mapToLong(DataFileValue::getSize).sum();
-      return new Size(tm.getExtent(), size);
-    }).iterator();
-  }
-
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/util/Retry.java b/core/src/main/java/org/apache/accumulo/core/util/Retry.java
index 952d980..916537d 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/Retry.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/Retry.java
@@ -18,11 +18,9 @@
  */
 package org.apache.accumulo.core.util;
 
-import static java.util.concurrent.TimeUnit.MILLISECONDS;
-import static java.util.concurrent.TimeUnit.NANOSECONDS;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 
-import java.security.SecureRandom;
-import java.util.concurrent.TimeUnit;
+import java.time.Duration;
 
 import org.slf4j.Logger;
 
@@ -35,18 +33,17 @@
  */
 public class Retry {
   private long maxRetries; // not final for testing
-  private long waitIncrement; // not final for testing
-  private long maxWait; // not final for testing
-  private final long logIntervalNanoSec;
+  private Duration waitIncrement; // not final for testing
+  private Duration maxWait; // not final for testing
+  private final Duration logInterval;
   private double backOffFactor;
   private long retriesDone;
-  private long currentWait;
-  private long initialWait;
+  private Duration currentWait;
+  private Duration initialWait;
 
   private boolean hasNeverLogged;
   private boolean hasLoggedWarn = false;
   private long lastRetryLog;
-  private static final SecureRandom random = new SecureRandom();
   private double currentBackOffFactor;
   private boolean doTimeJitter = true;
 
@@ -57,15 +54,15 @@
    * @param waitIncrement The amount of time (ms) to increment next wait time by
    * @param logInterval The amount of time (ms) between logging retries
    */
-  private Retry(long maxRetries, long startWait, long waitIncrement, long maxWait, long logInterval,
-      double backOffFactor) {
+  private Retry(long maxRetries, Duration startWait, Duration waitIncrement, Duration maxWait,
+      Duration logInterval, double backOffFactor) {
     this.maxRetries = maxRetries;
     this.maxWait = maxWait;
     this.waitIncrement = waitIncrement;
     this.retriesDone = 0;
     this.currentWait = startWait;
     this.initialWait = startWait;
-    this.logIntervalNanoSec = MILLISECONDS.toNanos(logInterval);
+    this.logInterval = logInterval;
     this.hasNeverLogged = true;
     this.lastRetryLog = -1;
     this.backOffFactor = backOffFactor;
@@ -94,19 +91,19 @@
 
   // Visible for testing
   @VisibleForTesting
-  long getCurrentWait() {
+  Duration getCurrentWait() {
     return currentWait;
   }
 
   // Visible for testing
   @VisibleForTesting
-  long getWaitIncrement() {
+  Duration getWaitIncrement() {
     return waitIncrement;
   }
 
   // Visible for testing
   @VisibleForTesting
-  long getMaxWait() {
+  Duration getMaxWait() {
     return maxWait;
   }
 
@@ -118,20 +115,20 @@
 
   // Visible for testing
   @VisibleForTesting
-  void setStartWait(long startWait) {
+  void setStartWait(Duration startWait) {
     this.currentWait = startWait;
     this.initialWait = startWait;
   }
 
   // Visible for testing
   @VisibleForTesting
-  void setWaitIncrement(long waitIncrement) {
+  void setWaitIncrement(Duration waitIncrement) {
     this.waitIncrement = waitIncrement;
   }
 
   // Visible for testing
   @VisibleForTesting
-  void setMaxWait(long maxWait) {
+  void setMaxWait(Duration maxWait) {
     this.maxWait = maxWait;
   }
 
@@ -145,8 +142,8 @@
     return maxRetries < 0;
   }
 
-  public long getLogInterval() {
-    return NANOSECONDS.toMillis(logIntervalNanoSec);
+  public Duration getLogInterval() {
+    return logInterval;
   }
 
   public boolean canRetry() {
@@ -171,29 +168,35 @@
   public void waitForNextAttempt(Logger log, String operationDescription)
       throws InterruptedException {
 
-    double waitFactor = (1 + (random.nextDouble() - 0.5) / 10.0) * currentBackOffFactor;
+    double waitFactor = (1 + (RANDOM.get().nextDouble() - 0.5) / 10.0) * currentBackOffFactor;
     if (!doTimeJitter) {
       waitFactor = currentBackOffFactor;
     }
     currentBackOffFactor = currentBackOffFactor * backOffFactor;
 
-    log.debug("Sleeping for {}ms before retrying operation : {} ", currentWait,
+    log.debug("Sleeping for {}ms before retrying operation : {} ", currentWait.toMillis(),
         operationDescription);
 
     sleep(currentWait);
 
     if (backOffFactor == 1) {
-      currentWait = Math.min(maxWait, currentWait + waitIncrement);
+      currentWait = currentWait.plus(waitIncrement);
+      if (currentWait.compareTo(maxWait) > 0) {
+        currentWait = maxWait;
+      }
     } else if (backOffFactor > 1.0) {
-      if (currentWait < maxWait) {
-        waitIncrement = (long) Math.ceil(waitFactor * this.initialWait);
-        currentWait = Math.min(maxWait, initialWait + waitIncrement);
+      waitIncrement = Duration.ofMillis((long) Math.ceil(waitFactor * initialWait.toMillis()));
+      Duration tempWait = initialWait.plus(waitIncrement);
+      if (tempWait.compareTo(maxWait) > 0) {
+        currentWait = maxWait;
+      } else {
+        currentWait = tempWait;
       }
     }
   }
 
-  protected void sleep(long wait) throws InterruptedException {
-    Thread.sleep(wait);
+  protected void sleep(Duration wait) throws InterruptedException {
+    Thread.sleep(wait.toMillis());
   }
 
   public void logRetry(Logger log, String message, Throwable t) {
@@ -205,7 +208,7 @@
       }
       hasNeverLogged = false;
       lastRetryLog = now;
-    } else if ((now - lastRetryLog) > logIntervalNanoSec) {
+    } else if ((now - lastRetryLog) > logInterval.toNanos()) {
       log.warn(getMessage(message), t);
       lastRetryLog = now;
       hasLoggedWarn = true;
@@ -225,7 +228,7 @@
       }
       hasNeverLogged = false;
       lastRetryLog = now;
-    } else if ((now - lastRetryLog) > logIntervalNanoSec) {
+    } else if ((now - lastRetryLog) > logInterval.toNanos()) {
       log.warn(getMessage(message));
       lastRetryLog = now;
       hasLoggedWarn = true;
@@ -269,33 +272,37 @@
      * @return this builder with the maximum number of retries set to the provided value
      */
     NeedsRetryDelay maxRetries(long max);
+
+    /**
+     * @return this builder with the maximum number of retries set to the number of retries that can
+     *         occur within the given duration
+     */
+    NeedsRetryDelay maxRetriesWithinDuration(Duration duration);
   }
 
   public interface NeedsRetryDelay {
     /**
-     * @param duration the amount of time to wait before the first retry; input is converted to
-     *        milliseconds, rounded down to the nearest
+     * @param duration the amount of time to wait before the first retry
      * @return this builder with the initial wait period set
      */
-    NeedsTimeIncrement retryAfter(long duration, TimeUnit unit);
+    NeedsTimeIncrement retryAfter(Duration duration);
   }
 
   public interface NeedsTimeIncrement {
     /**
-     * @param duration the amount of additional time to add before each subsequent retry; input is
-     *        converted to milliseconds, rounded down to the nearest
+     * @param duration the amount of additional time to add before each subsequent retry
      * @return this builder with the increment amount set
      */
-    NeedsMaxWait incrementBy(long duration, TimeUnit unit);
+    NeedsMaxWait incrementBy(Duration duration);
   }
 
   public interface NeedsMaxWait {
     /**
      * @param duration the maximum amount of time to which the waiting period between retries can be
-     *        incremented; input is converted to milliseconds, rounded down to the nearest
+     *        incremented
      * @return this builder with a maximum time limit set
      */
-    NeedsBackOffFactor maxWait(long duration, TimeUnit unit);
+    NeedsBackOffFactor maxWait(Duration duration);
   }
 
   public interface NeedsBackOffFactor {
@@ -309,11 +316,10 @@
 
   public interface NeedsLogInterval {
     /**
-     * @param duration the minimum time interval between logging that a retry is occurring; input is
-     *        converted to milliseconds, rounded down to the nearest
+     * @param duration the minimum time interval between logging that a retry is occurring
      * @return this builder with a logging interval set
      */
-    BuilderDone logInterval(long duration, TimeUnit unit);
+    BuilderDone logInterval(Duration duration);
   }
 
   public interface BuilderDone {
@@ -353,11 +359,12 @@
 
     private boolean modifiable = true;
     private long maxRetries;
-    private long initialWait;
-    private long maxWait;
-    private long waitIncrement;
-    private long logInterval;
+    private Duration initialWait;
+    private Duration maxWait;
+    private Duration waitIncrement;
+    private Duration logInterval;
     private double backOffFactor = 1.5;
+    private Duration retriesForDuration = null;
 
     RetryFactoryBuilder() {}
 
@@ -382,19 +389,62 @@
     }
 
     @Override
-    public NeedsTimeIncrement retryAfter(long duration, TimeUnit unit) {
+    public NeedsRetryDelay maxRetriesWithinDuration(Duration duration) {
       checkState();
-      Preconditions.checkArgument(duration >= 0, "Initial waiting period must not be negative");
-      this.initialWait = unit.toMillis(duration);
+      Preconditions.checkArgument(!duration.isNegative(),
+          "Duration for retries must not be negative");
+      this.retriesForDuration = duration;
+      return this;
+    }
+
+    /**
+     * Calculate the maximum number of retries that can occur within {@link #retriesForDuration}
+     */
+    private void calculateRetriesWithinDuration() {
+      long numberOfRetries = 0;
+      long cumulativeWaitTimeMillis = 0;
+      long currentWaitTimeMillis = initialWait.toMillis();
+      final long retriesForDurationMillis = retriesForDuration.toMillis();
+
+      // set an upper bound for the number of retries
+      final long maxRetries = Duration.ofHours(1).toMillis();
+
+      while (cumulativeWaitTimeMillis + currentWaitTimeMillis <= retriesForDurationMillis
+          && numberOfRetries < maxRetries) {
+
+        cumulativeWaitTimeMillis += currentWaitTimeMillis;
+        numberOfRetries++;
+
+        if (backOffFactor > 1.0) {
+          currentWaitTimeMillis = (long) Math.ceil(currentWaitTimeMillis * backOffFactor);
+        } else {
+          currentWaitTimeMillis += waitIncrement.toMillis();
+        }
+
+        if (currentWaitTimeMillis > maxWait.toMillis()) {
+          currentWaitTimeMillis = maxWait.toMillis(); // Ensure wait time does not exceed maxWait
+        }
+
+      }
+
+      this.maxRetries = numberOfRetries;
+    }
+
+    @Override
+    public NeedsTimeIncrement retryAfter(Duration duration) {
+      checkState();
+      Preconditions.checkArgument(!duration.isNegative(),
+          "Initial waiting period must not be negative");
+      this.initialWait = duration;
       return this;
     }
 
     @Override
-    public NeedsMaxWait incrementBy(long duration, TimeUnit unit) {
+    public NeedsMaxWait incrementBy(Duration duration) {
       checkState();
-      Preconditions.checkArgument(duration >= 0,
+      Preconditions.checkArgument(!duration.isNegative(),
           "Amount of time to increment the wait between each retry must not be negative");
-      this.waitIncrement = unit.toMillis(duration);
+      this.waitIncrement = duration;
       return this;
     }
 
@@ -408,20 +458,20 @@
     }
 
     @Override
-    public NeedsBackOffFactor maxWait(long duration, TimeUnit unit) {
+    public NeedsBackOffFactor maxWait(Duration duration) {
       checkState();
-      this.maxWait = unit.toMillis(duration);
-      Preconditions.checkArgument(maxWait >= initialWait,
+      this.maxWait = duration;
+      Preconditions.checkArgument(maxWait.compareTo(initialWait) >= 0,
           "Maximum wait between retries must not be less than the initial delay");
       return this;
     }
 
     @Override
-    public BuilderDone logInterval(long duration, TimeUnit unit) {
+    public BuilderDone logInterval(Duration duration) {
       checkState();
-      Preconditions.checkArgument(duration >= 0,
+      Preconditions.checkArgument(!duration.isNegative(),
           "The amount of time between logging retries must not be negative");
-      this.logInterval = unit.toMillis(duration);
+      this.logInterval = duration;
       return this;
     }
 
@@ -433,6 +483,10 @@
 
     @Override
     public Retry createRetry() {
+      if (retriesForDuration != null) {
+        calculateRetriesWithinDuration();
+      }
+      this.modifiable = false;
       return new Retry(maxRetries, initialWait, waitIncrement, maxWait, logInterval, backOffFactor);
     }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/util/RowRangeUtil.java b/core/src/main/java/org/apache/accumulo/core/util/RowRangeUtil.java
new file mode 100644
index 0000000..d61e17b
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/util/RowRangeUtil.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.util;
+
+import org.apache.accumulo.core.data.ByteSequence;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.dataImpl.KeyExtent;
+
+import com.google.common.base.Preconditions;
+
+public class RowRangeUtil {
+
+  /**
+   * Validates a range is a valid KeyExtent data range, which is a special type of row range. These
+   * ranges are created by calling {@code new Range(startRow, false, endRow, true);} which is what
+   * {@link KeyExtent#toDataRange()} does.
+   *
+   * A KeyExtent data row range is defined as:
+   * <ul>
+   * <li>A range that has an inclusive start and exclusive end</li>
+   * <li>A range that only has the row portion set</li>
+   * <li>A range where both the start and end key end with a zero byte</li>
+   * </ul>
+   *
+   * @param range The range to validate
+   * @return The original range
+   */
+  public static Range requireKeyExtentDataRange(Range range) {
+    String errorMsg = "Range is not a KeyExtent data range";
+
+    if (!range.isInfiniteStartKey()) {
+      Preconditions.checkArgument(range.isStartKeyInclusive(),
+          "%s, start key must be inclusive. %s", errorMsg, range);
+      Preconditions.checkArgument(isOnlyRowSet(range.getStartKey()),
+          "%s, start key must only contain a row. %s", errorMsg, range);
+      Preconditions.checkArgument(isRowSuffixZeroByte(range.getStartKey()),
+          "%s, start key does not end with zero byte. %s, ", errorMsg, range);
+    }
+
+    if (!range.isInfiniteStopKey()) {
+      Preconditions.checkArgument(!range.isEndKeyInclusive(), "%s, end key must be exclusive. %s",
+          errorMsg, range);
+      Preconditions.checkArgument(isOnlyRowSet(range.getEndKey()),
+          "%s, end key must only contain a row. %s", errorMsg, range);
+      Preconditions.checkArgument(isRowSuffixZeroByte(range.getEndKey()),
+          "%s, end key does not end with a zero byte. %s, ", errorMsg, range);
+    }
+
+    return range;
+  }
+
+  public static boolean isOnlyRowSet(Key key) {
+    return key.getColumnFamilyData().length() == 0 && key.getColumnQualifierData().length() == 0
+        && key.getColumnVisibilityData().length() == 0 && key.getTimestamp() == Long.MAX_VALUE;
+  }
+
+  public static boolean isRowSuffixZeroByte(Key key) {
+    var row = key.getRowData();
+    return row.length() > 0 && row.byteAt(row.length() - 1) == (byte) 0x00;
+  }
+
+  public static ByteSequence stripZeroTail(ByteSequence row) {
+    if (row.byteAt(row.length() - 1) == (byte) 0x00) {
+      return row.subSequence(0, row.length() - 1);
+    }
+    return row;
+  }
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/util/ServerServices.java b/core/src/main/java/org/apache/accumulo/core/util/ServerServices.java
deleted file mode 100644
index fd90077..0000000
--- a/core/src/main/java/org/apache/accumulo/core/util/ServerServices.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.util;
-
-import java.util.EnumMap;
-
-public class ServerServices implements Comparable<ServerServices> {
-  public static enum Service {
-    TSERV_CLIENT, GC_CLIENT, COMPACTOR_CLIENT, SSERV_CLIENT;
-  }
-
-  public static final String SERVICE_SEPARATOR = ";";
-  public static final String SEPARATOR_CHAR = "=";
-
-  private EnumMap<Service,String> services;
-  private String stringForm = null;
-
-  public ServerServices(String services) {
-    this.services = new EnumMap<>(Service.class);
-
-    String[] addresses = services.split(SERVICE_SEPARATOR);
-    for (String address : addresses) {
-      String[] sa = address.split(SEPARATOR_CHAR, 2);
-      this.services.put(Service.valueOf(sa[0]), sa[1]);
-    }
-  }
-
-  public ServerServices(String address, Service service) {
-    this(service.name() + SEPARATOR_CHAR + address);
-  }
-
-  public String getAddressString(Service service) {
-    return services.get(service);
-  }
-
-  public HostAndPort getAddress(Service service) {
-    return AddressUtil.parseAddress(getAddressString(service), false);
-  }
-
-  // DON'T CHANGE THIS; WE'RE USING IT FOR SERIALIZATION!!!
-  @Override
-  public String toString() {
-    if (stringForm == null) {
-      StringBuilder sb = new StringBuilder();
-      String prefix = "";
-      for (Service service : new Service[] {Service.TSERV_CLIENT, Service.GC_CLIENT}) {
-        if (services.containsKey(service)) {
-          sb.append(prefix).append(service.name()).append(SEPARATOR_CHAR)
-              .append(services.get(service));
-          prefix = SERVICE_SEPARATOR;
-        }
-      }
-      stringForm = sb.toString();
-    }
-    return stringForm;
-  }
-
-  @Override
-  public int hashCode() {
-    return toString().hashCode();
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (o instanceof ServerServices) {
-      return toString().equals(o.toString());
-    }
-    return false;
-  }
-
-  @Override
-  public int compareTo(ServerServices other) {
-    return toString().compareTo(other.toString());
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/util/UtilWaitThread.java b/core/src/main/java/org/apache/accumulo/core/util/UtilWaitThread.java
index a05e1d2..1ad269a 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/UtilWaitThread.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/UtilWaitThread.java
@@ -18,10 +18,6 @@
  */
 package org.apache.accumulo.core.util;
 
-import static java.util.concurrent.TimeUnit.NANOSECONDS;
-
-import java.util.concurrent.TimeUnit;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -35,36 +31,4 @@
       log.error("{}", e.getMessage(), e);
     }
   }
-
-  /**
-   * Copied from Guava release 23. The Uniterruptibles class was annotated as Beta by Google,
-   * therefore unstable to use. The following javadoc was copied from
-   * com.google.common.util.concurrent.Uninterruptibles:
-   *
-   * Utilities for treating interruptible operations as uninterruptible. In all cases, if a thread
-   * is interrupted during such a call, the call continues to block until the result is available or
-   * the timeout elapses, and only then re-interrupts the thread.
-   *
-   */
-  public static void sleepUninterruptibly(long sleepFor, TimeUnit unit) {
-    boolean interrupted = false;
-    try {
-      long remainingNanos = unit.toNanos(sleepFor);
-      long end = System.nanoTime() + remainingNanos;
-      while (true) {
-        try {
-          // TimeUnit.sleep() treats negative timeouts just like zero.
-          NANOSECONDS.sleep(remainingNanos);
-          return;
-        } catch (InterruptedException e) {
-          interrupted = true;
-          remainingNanos = end - System.nanoTime();
-        }
-      }
-    } finally {
-      if (interrupted) {
-        Thread.currentThread().interrupt();
-      }
-    }
-  }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/util/UuidUtil.java b/core/src/main/java/org/apache/accumulo/core/util/UuidUtil.java
deleted file mode 100644
index 83d41ab..0000000
--- a/core/src/main/java/org/apache/accumulo/core/util/UuidUtil.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.util;
-
-public class UuidUtil {
-  /**
-   * A fast method for verifying a suffix of a string looks like a uuid.
-   *
-   * @param offset location where the uuid starts. Its expected the uuid occupies the rest of the
-   *        string.
-   */
-  public static boolean isUUID(String uuid, int offset) {
-    if (uuid.length() - offset != 36) {
-      return false;
-    }
-    for (int i = 0; i < 36; i++) {
-      var c = uuid.charAt(i + offset);
-      if (i == 8 || i == 13 || i == 18 || i == 23) {
-        if (c != '-') {
-          // expect '-' char at above positions, did not see it
-          return false;
-        }
-      } else if (c < '0' || (c > '9' && c < 'A') || (c > 'F' && c < 'a') || c > 'f') {
-        // expected hex at all other positions, did not see hex chars
-        return false;
-      }
-    }
-    return true;
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/util/Validators.java b/core/src/main/java/org/apache/accumulo/core/util/Validators.java
index 3c756ea..084c912 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/Validators.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/Validators.java
@@ -24,8 +24,7 @@
 
 import org.apache.accumulo.core.clientImpl.Namespace;
 import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.util.tables.TableNameUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -155,7 +154,8 @@
   public static final Validator<String> NEW_TABLE_NAME =
       new Validator<>(tableName -> _tableName(tableName, false));
 
-  private static final List<String> metadataTables = List.of(RootTable.NAME, MetadataTable.NAME);
+  private static final List<String> metadataTables =
+      List.of(AccumuloTable.ROOT.tableName(), AccumuloTable.METADATA.tableName());
   public static final Validator<String> NOT_METADATA_TABLE = new Validator<>(t -> {
     if (t == null) {
       return NameSegment.Table.isNull();
@@ -192,9 +192,7 @@
     if (id == null) {
       return Optional.of("Table id must not be null");
     }
-    @SuppressWarnings("deprecation")
-    TableId replicationId = org.apache.accumulo.core.replication.ReplicationTable.ID;
-    if (RootTable.ID.equals(id) || MetadataTable.ID.equals(id) || replicationId.equals(id)
+    if (AccumuloTable.ROOT.tableId().equals(id) || AccumuloTable.METADATA.tableId().equals(id)
         || VALID_ID_PATTERN.matcher(id.canonical()).matches()) {
       return Validator.OK;
     }
@@ -206,12 +204,13 @@
     if (id == null) {
       return Optional.of("Table id must not be null");
     }
-    if (id.equals(MetadataTable.ID)) {
-      return Optional.of("Cloning " + MetadataTable.NAME + " is dangerous and no longer supported,"
-          + " see https://github.com/apache/accumulo/issues/1309.");
+    if (id.equals(AccumuloTable.METADATA.tableId())) {
+      return Optional.of(
+          "Cloning " + AccumuloTable.METADATA.tableName() + " is dangerous and no longer supported,"
+              + " see https://github.com/apache/accumulo/issues/1309.");
     }
-    if (id.equals(RootTable.ID)) {
-      return Optional.of("Unable to clone " + RootTable.NAME);
+    if (id.equals(AccumuloTable.ROOT.tableId())) {
+      return Optional.of("Unable to clone " + AccumuloTable.ROOT.tableName());
     }
     return Validator.OK;
   });
@@ -220,9 +219,9 @@
     if (id == null) {
       return Optional.of("Table id must not be null");
     }
-    if (RootTable.ID.equals(id)) {
-      return Optional
-          .of("Table must not be the " + RootTable.NAME + "(Id: " + RootTable.ID + ") table");
+    if (AccumuloTable.ROOT.tableId().equals(id)) {
+      return Optional.of("Table must not be the " + AccumuloTable.ROOT.tableName() + "(Id: "
+          + AccumuloTable.ROOT.tableId() + ") table");
     }
     return Validator.OK;
   });
diff --git a/core/src/main/java/org/apache/accumulo/core/util/compaction/CompactionJobPrioritizer.java b/core/src/main/java/org/apache/accumulo/core/util/compaction/CompactionJobPrioritizer.java
index 1f87459..039201b 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/compaction/CompactionJobPrioritizer.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/compaction/CompactionJobPrioritizer.java
@@ -29,13 +29,13 @@
       Comparator.comparingInt(CompactionJob::getPriority)
           .thenComparingInt(job -> job.getFiles().size()).reversed();
 
+  @SuppressWarnings("deprecation")
   public static short createPriority(CompactionKind kind, int totalFiles, int compactingFiles) {
 
     int prio = totalFiles + compactingFiles;
 
     switch (kind) {
       case USER:
-      case CHOP:
         // user-initiated compactions will have a positive priority
         // based on number of files
         if (prio > Short.MAX_VALUE) {
diff --git a/core/src/main/java/org/apache/accumulo/core/util/compaction/CompactionPlannerInitParams.java b/core/src/main/java/org/apache/accumulo/core/util/compaction/CompactionPlannerInitParams.java
index eb28715..86fd8c4 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/compaction/CompactionPlannerInitParams.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/compaction/CompactionPlannerInitParams.java
@@ -23,7 +23,6 @@
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.spi.common.ServiceEnvironment;
 import org.apache.accumulo.core.spi.compaction.CompactionExecutorId;
 import org.apache.accumulo.core.spi.compaction.CompactionPlanner;
@@ -38,14 +37,16 @@
   private final Set<CompactionExecutorId> requestedExternalExecutors;
   private final ServiceEnvironment senv;
   private final CompactionServiceId serviceId;
+  private final String prefix;
 
-  public CompactionPlannerInitParams(CompactionServiceId serviceId, Map<String,String> plannerOpts,
-      ServiceEnvironment senv) {
+  public CompactionPlannerInitParams(CompactionServiceId serviceId, String prefix,
+      Map<String,String> plannerOpts, ServiceEnvironment senv) {
     this.serviceId = serviceId;
     this.plannerOpts = plannerOpts;
     this.requestedExecutors = new HashMap<>();
     this.requestedExternalExecutors = new HashSet<>();
     this.senv = senv;
+    this.prefix = prefix;
   }
 
   @Override
@@ -60,7 +61,7 @@
 
   @Override
   public String getFullyQualifiedOption(String key) {
-    return Property.TSERV_COMPACTION_SERVICE_PREFIX.getKey() + serviceId + ".planner.opts." + key;
+    return prefix + serviceId + ".planner.opts." + key;
   }
 
   @Override
diff --git a/core/src/main/java/org/apache/accumulo/core/util/compaction/CompactionServicesConfig.java b/core/src/main/java/org/apache/accumulo/core/util/compaction/CompactionServicesConfig.java
index 71d4490..5e2f233 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/compaction/CompactionServicesConfig.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/compaction/CompactionServicesConfig.java
@@ -21,13 +21,14 @@
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Objects;
-import java.util.function.Consumer;
+import java.util.stream.Collectors;
 
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.ConfigurationTypeHelper;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.spi.compaction.CompactionServiceId;
-import org.apache.accumulo.core.spi.compaction.DefaultCompactionPlanner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.collect.Sets;
 
@@ -35,110 +36,112 @@
  * This class serves to configure compaction services from an {@link AccumuloConfiguration} object.
  *
  * Specifically, compaction service properties (those prefixed by "tserver.compaction.major
- * .service") are used.
+ * .service" or "compaction.service") are used.
  */
 public class CompactionServicesConfig {
 
+  private static final Logger log = LoggerFactory.getLogger(CompactionServicesConfig.class);
   private final Map<String,String> planners = new HashMap<>();
+  private final Map<String,String> plannerPrefixes = new HashMap<>();
   private final Map<String,Long> rateLimits = new HashMap<>();
   private final Map<String,Map<String,String>> options = new HashMap<>();
+  @SuppressWarnings("deprecation")
+  private final Property oldPrefix = Property.TSERV_COMPACTION_SERVICE_PREFIX;
+  private final Property newPrefix = Property.COMPACTION_SERVICE_PREFIX;
   long defaultRateLimit;
-  private final Consumer<String> deprecationWarningConsumer;
 
   public static final CompactionServiceId DEFAULT_SERVICE = CompactionServiceId.of("default");
 
-  @SuppressWarnings("removal")
-  private long getDefaultThroughput(AccumuloConfiguration aconf) {
-    if (aconf.isPropertySet(Property.TSERV_MAJC_THROUGHPUT)) {
-      return aconf.getAsBytes(Property.TSERV_MAJC_THROUGHPUT);
-    }
-
+  @SuppressWarnings("deprecation")
+  private long getDefaultThroughput() {
     return ConfigurationTypeHelper
         .getMemoryAsBytes(Property.TSERV_COMPACTION_SERVICE_DEFAULT_RATE_LIMIT.getDefaultValue());
   }
 
-  @SuppressWarnings("removal")
-  private Map<String,String> getConfiguration(AccumuloConfiguration aconf) {
+  private Map<String,Map<String,String>> getConfiguration(AccumuloConfiguration aconf) {
+    Map<String,Map<String,String>> properties = new HashMap<>();
 
-    Map<String,String> configs =
-        aconf.getAllPropertiesWithPrefix(Property.TSERV_COMPACTION_SERVICE_PREFIX);
+    var newProps = aconf.getAllPropertiesWithPrefixStripped(newPrefix);
+    properties.put(newPrefix.getKey(), newProps);
 
-    // check if deprecated properties for compaction executor are set
-    if (aconf.isPropertySet(Property.TSERV_MAJC_MAXCONCURRENT)) {
+    // get all of the services under the new prefix
+    var newServices =
+        newProps.keySet().stream().map(prop -> prop.split("\\.")[0]).collect(Collectors.toSet());
 
-      String defaultServicePrefix =
-          Property.TSERV_COMPACTION_SERVICE_PREFIX.getKey() + DEFAULT_SERVICE.canonical() + ".";
+    Map<String,String> oldServices = new HashMap<>();
 
-      // check if any properties for the default compaction service are set
-      boolean defaultServicePropsSet = configs.keySet().stream()
-          .filter(key -> key.startsWith(defaultServicePrefix)).map(Property::getPropertyByKey)
-          .anyMatch(prop -> prop == null || aconf.isPropertySet(prop));
-
-      if (defaultServicePropsSet) {
-
-        String warning = "The deprecated property " + Property.TSERV_MAJC_MAXCONCURRENT.getKey()
-            + " was set. Properties with the prefix " + defaultServicePrefix
-            + " were also set which replace the deprecated properties. The deprecated property "
-            + "was therefore ignored.";
-
-        deprecationWarningConsumer.accept(warning);
-
+    for (Map.Entry<String,String> entry : aconf.getAllPropertiesWithPrefixStripped(oldPrefix)
+        .entrySet()) {
+      // Discard duplicate service definitions
+      var service = entry.getKey().split("\\.")[0];
+      if (newServices.contains(service)) {
+        log.warn("Duplicate compaction service '{}' definition exists. Ignoring property : '{}'",
+            service, entry.getKey());
       } else {
-        String numThreads = aconf.get(Property.TSERV_MAJC_MAXCONCURRENT);
-
-        // Its possible a user has configured the other compaction services, but not the default
-        // service. In this case want to produce a config with the default service configs
-        // overridden using deprecated configs.
-
-        HashMap<String,String> configsCopy = new HashMap<>(configs);
-
-        Map<String,String> defaultServiceConfigs =
-            Map.of(defaultServicePrefix + "planner", DefaultCompactionPlanner.class.getName(),
-                defaultServicePrefix + "planner.opts.executors",
-                "[{'name':'deprecated', 'numThreads':" + numThreads + "}]");
-
-        configsCopy.putAll(defaultServiceConfigs);
-
-        String warning = "The deprecated property " + Property.TSERV_MAJC_MAXCONCURRENT.getKey()
-            + " was set. Properties with the prefix " + defaultServicePrefix
-            + " were not set, these should replace the deprecated properties. The old properties "
-            + "were automatically mapped to the new properties in process creating : "
-            + defaultServiceConfigs + ".";
-
-        deprecationWarningConsumer.accept(warning);
-
-        configs = Map.copyOf(configsCopy);
+        oldServices.put(entry.getKey(), entry.getValue());
       }
     }
-
-    return configs;
-
+    properties.put(oldPrefix.getKey(), oldServices);
+    // Return unmodifiable map
+    return Map.copyOf(properties);
   }
 
-  public CompactionServicesConfig(AccumuloConfiguration aconf,
-      Consumer<String> deprecationWarningConsumer) {
-    this.deprecationWarningConsumer = deprecationWarningConsumer;
-    Map<String,String> configs = getConfiguration(aconf);
+  public CompactionServicesConfig(AccumuloConfiguration aconf) {
+    Map<String,Map<String,String>> configs = getConfiguration(aconf);
 
-    configs.forEach((prop, val) -> {
-
-      var suffix = prop.substring(Property.TSERV_COMPACTION_SERVICE_PREFIX.getKey().length());
-      String[] tokens = suffix.split("\\.");
-      if (tokens.length == 4 && tokens[1].equals("planner") && tokens[2].equals("opts")) {
-        options.computeIfAbsent(tokens[0], k -> new HashMap<>()).put(tokens[3], val);
-      } else if (tokens.length == 2 && tokens[1].equals("planner")) {
-        planners.put(tokens[0], val);
-      } else if (tokens.length == 3 && tokens[1].equals("rate") && tokens[2].equals("limit")) {
-        var eprop = Property.getPropertyByKey(prop);
-        if (eprop == null || aconf.isPropertySet(eprop) || !isDeprecatedThroughputSet(aconf)) {
-          rateLimits.put(tokens[0], ConfigurationTypeHelper.getFixedMemoryAsBytes(val));
+    // Find compaction planner defs first.
+    configs.forEach((prefix, props) -> {
+      props.forEach((prop, val) -> {
+        String[] tokens = prop.split("\\.");
+        if (tokens.length == 2 && tokens[1].equals("planner")) {
+          if (prefix.equals(oldPrefix.getKey())) {
+            // Log a warning if the old prefix planner is defined by a user.
+            Property userDefined = null;
+            try {
+              userDefined = Property.valueOf(prefix + prop);
+            } catch (IllegalArgumentException e) {
+              log.trace("Property: {} is not set by default configuration", prefix + prop);
+            }
+            boolean isPropSet = true;
+            if (userDefined != null) {
+              isPropSet = aconf.isPropertySet(userDefined);
+            }
+            if (isPropSet) {
+              log.warn(
+                  "Found compaction planner '{}' using a deprecated prefix. Please update property to use the '{}' prefix",
+                  tokens[0], newPrefix);
+            }
+          }
+          plannerPrefixes.put(tokens[0], prefix);
+          planners.put(tokens[0], val);
         }
-      } else {
-        throw new IllegalArgumentException("Malformed compaction service property " + prop);
-      }
+      });
     });
 
-    defaultRateLimit = getDefaultThroughput(aconf);
+    // Now find all compaction planner options.
+    configs.forEach((prefix, props) -> {
+      props.forEach((prop, val) -> {
+        String[] tokens = prop.split("\\.");
+        if (!plannerPrefixes.containsKey(tokens[0])) {
+          throw new IllegalArgumentException(
+              "Incomplete compaction service definition, missing planner class: " + prop);
+        }
+        if (tokens.length == 4 && tokens[1].equals("planner") && tokens[2].equals("opts")) {
+          options.computeIfAbsent(tokens[0], k -> new HashMap<>()).put(tokens[3], val);
+        } else if (tokens.length == 3 && tokens[1].equals("rate") && tokens[2].equals("limit")) {
+          var eprop = Property.getPropertyByKey(prop);
+          if (eprop == null || aconf.isPropertySet(eprop)) {
+            rateLimits.put(tokens[0], ConfigurationTypeHelper.getFixedMemoryAsBytes(val));
+          }
+        } else if (tokens.length == 2 && tokens[1].equals("planner")) {
+          return; // moves to next opt
+        } else {
+          throw new IllegalArgumentException(
+              "Malformed compaction service property " + prefix + prop);
+        }
+      });
+    });
+    defaultRateLimit = getDefaultThroughput();
 
     var diff = Sets.difference(options.keySet(), planners.keySet());
 
@@ -149,11 +152,6 @@
 
   }
 
-  @SuppressWarnings("removal")
-  private boolean isDeprecatedThroughputSet(AccumuloConfiguration aconf) {
-    return aconf.isPropertySet(Property.TSERV_MAJC_THROUGHPUT);
-  }
-
   public long getRateLimit(String serviceName) {
     return getRateLimits().getOrDefault(serviceName, defaultRateLimit);
   }
@@ -178,6 +176,10 @@
     return planners;
   }
 
+  public String getPlannerPrefix(String service) {
+    return plannerPrefixes.get(service);
+  }
+
   public Map<String,Long> getRateLimits() {
     return rateLimits;
   }
diff --git a/core/src/main/thrift/trace.thrift b/core/src/main/java/org/apache/accumulo/core/util/compaction/DeprecatedCompactionKind.java
similarity index 68%
rename from core/src/main/thrift/trace.thrift
rename to core/src/main/java/org/apache/accumulo/core/util/compaction/DeprecatedCompactionKind.java
index 2950b34..8b814b2 100644
--- a/core/src/main/thrift/trace.thrift
+++ b/core/src/main/java/org/apache/accumulo/core/util/compaction/DeprecatedCompactionKind.java
@@ -16,13 +16,12 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-namespace java org.apache.accumulo.core.trace.thrift
-namespace cpp org.apache.accumulo.core.trace.thrift
+package org.apache.accumulo.core.util.compaction;
 
-# OpenTelemetry uses the standards at https://www.w3.org/TR/trace-context/
-# to propagate information across process boundaries.
-struct TInfo {
-  // 1:i64 traceId - removed in 2.1.0
-  // 2:i64 parentId - removed in 2.1.0  
-  3:map<string,string> headers
+import org.apache.accumulo.core.spi.compaction.CompactionKind;
+
+// The purpose of this class is to minimize the amount of code that has to suppress deprecation warnings.
+public class DeprecatedCompactionKind {
+  @SuppressWarnings("deprecation")
+  public static final CompactionKind SELECTOR = CompactionKind.SELECTOR;
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/util/compaction/ExternalCompactionUtil.java b/core/src/main/java/org/apache/accumulo/core/util/compaction/ExternalCompactionUtil.java
index c2e5f81..bbcc3f1 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/compaction/ExternalCompactionUtil.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/compaction/ExternalCompactionUtil.java
@@ -18,8 +18,6 @@
  */
 package org.apache.accumulo.core.util.compaction;
 
-import static java.nio.charset.StandardCharsets.UTF_8;
-
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
@@ -27,6 +25,7 @@
 import java.util.List;
 import java.util.Map;
 import java.util.Optional;
+import java.util.Set;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
@@ -36,16 +35,16 @@
 import org.apache.accumulo.core.clientImpl.ClientContext;
 import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.compaction.thrift.CompactorService;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock;
 import org.apache.accumulo.core.fate.zookeeper.ZooCache.ZcStat;
 import org.apache.accumulo.core.fate.zookeeper.ZooReader;
+import org.apache.accumulo.core.lock.ServiceLock;
+import org.apache.accumulo.core.lock.ServiceLockData.ThriftService;
 import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
 import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
 import org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction;
 import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob;
 import org.apache.accumulo.core.trace.TraceUtil;
-import org.apache.accumulo.core.util.HostAndPort;
 import org.apache.accumulo.core.util.threads.ThreadPools;
 import org.apache.thrift.TException;
 import org.apache.zookeeper.KeeperException;
@@ -53,6 +52,8 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.net.HostAndPort;
+
 public class ExternalCompactionUtil {
 
   private static class RunningCompactionFuture {
@@ -101,25 +102,21 @@
    */
   public static Optional<HostAndPort> findCompactionCoordinator(ClientContext context) {
     final String lockPath = context.getZooKeeperRoot() + Constants.ZCOORDINATOR_LOCK;
-    byte[] address =
-        ServiceLock.getLockData(context.getZooCache(), ServiceLock.path(lockPath), new ZcStat());
-    if (null == address) {
-      return Optional.empty();
-    }
-    return Optional.of(HostAndPort.fromString(new String(address, UTF_8)));
+    return ServiceLock.getLockData(context.getZooCache(), ServiceLock.path(lockPath), new ZcStat())
+        .map(sld -> sld.getAddress(ThriftService.COORDINATOR));
   }
 
   /**
    * @return map of queue names to compactor addresses
    */
-  public static Map<String,List<HostAndPort>> getCompactorAddrs(ClientContext context) {
+  public static Map<String,Set<HostAndPort>> getCompactorAddrs(ClientContext context) {
     try {
-      final Map<String,List<HostAndPort>> queuesAndAddresses = new HashMap<>();
+      final Map<String,Set<HostAndPort>> queuesAndAddresses = new HashMap<>();
       final String compactorQueuesPath = context.getZooKeeperRoot() + Constants.ZCOMPACTORS;
       ZooReader zooReader = context.getZooReader();
       List<String> queues = zooReader.getChildren(compactorQueuesPath);
       for (String queue : queues) {
-        queuesAndAddresses.putIfAbsent(queue, new ArrayList<>());
+        queuesAndAddresses.putIfAbsent(queue, new HashSet<>());
         try {
           List<String> compactors = zooReader.getChildren(compactorQueuesPath + "/" + queue);
           for (String compactor : compactors) {
@@ -139,10 +136,10 @@
 
       return queuesAndAddresses;
     } catch (KeeperException e) {
-      throw new RuntimeException(e);
+      throw new IllegalStateException(e);
     } catch (InterruptedException e) {
       Thread.currentThread().interrupt();
-      throw new RuntimeException(e);
+      throw new IllegalStateException(e);
     }
   }
 
@@ -242,7 +239,7 @@
           results.add(new RunningCompaction(job, compactorAddress, rcf.getQueue()));
         }
       } catch (InterruptedException | ExecutionException e) {
-        throw new RuntimeException(e);
+        throw new IllegalStateException(e);
       }
     });
     return results;
@@ -270,7 +267,7 @@
           runningIds.add(ceid);
         }
       } catch (InterruptedException | ExecutionException e) {
-        throw new RuntimeException(e);
+        throw new IllegalStateException(e);
       }
     });
 
diff --git a/core/src/main/java/org/apache/accumulo/core/util/format/BinaryFormatter.java b/core/src/main/java/org/apache/accumulo/core/util/format/BinaryFormatter.java
deleted file mode 100644
index 3dab96c..0000000
--- a/core/src/main/java/org/apache/accumulo/core/util/format/BinaryFormatter.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.util.format;
-
-import java.util.Map.Entry;
-
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.security.ColumnVisibility;
-
-/**
- * @deprecated since 1.8.0; Use {@link DefaultFormatter} providing showLength and printTimestamps
- *             via {@link FormatterConfig}.
- */
-@Deprecated(since = "1.8.0")
-public class BinaryFormatter extends DefaultFormatter {
-  // this class can probably be replaced by DefaultFormatter since DefaultFormatter has the max
-  // length stuff
-  @Override
-  public String next() {
-    checkState(true);
-    return formatEntry(getScannerIterator().next(), config.willPrintTimestamps(),
-        config.getShownLength());
-  }
-
-  public static String formatEntry(Entry<Key,Value> entry, boolean printTimestamps,
-      int shownLength) {
-    StringBuilder sb = new StringBuilder();
-
-    Key key = entry.getKey();
-
-    // append row
-    appendText(sb, key.getRow(), shownLength).append(" ");
-
-    // append column family
-    appendText(sb, key.getColumnFamily(), shownLength).append(":");
-
-    // append column qualifier
-    appendText(sb, key.getColumnQualifier(), shownLength).append(" ");
-
-    // append visibility expression
-    sb.append(new ColumnVisibility(key.getColumnVisibility()));
-
-    // append timestamp
-    if (printTimestamps) {
-      sb.append(" ").append(entry.getKey().getTimestamp());
-    }
-
-    // append value
-    Value value = entry.getValue();
-    if (value != null && value.getSize() > 0) {
-      sb.append("\t");
-      appendValue(sb, value, shownLength);
-    }
-    return sb.toString();
-  }
-
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/util/format/DateStringFormatter.java b/core/src/main/java/org/apache/accumulo/core/util/format/DateStringFormatter.java
deleted file mode 100644
index 9383e07..0000000
--- a/core/src/main/java/org/apache/accumulo/core/util/format/DateStringFormatter.java
+++ /dev/null
@@ -1,85 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.util.format;
-
-import java.util.Map.Entry;
-import java.util.TimeZone;
-
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Value;
-
-/**
- * This class is <strong>not</strong> recommended because
- * {@link #initialize(Iterable, FormatterConfig)} replaces parameters in {@link FormatterConfig},
- * which could surprise users.
- *
- * This class can be replaced by {@link DefaultFormatter} where FormatterConfig is initialized with
- * a DateFormat set to {@link #DATE_FORMAT}. See
- * {@link DateFormatSupplier#createSimpleFormatSupplier(String, java.util.TimeZone)}.
- *
- * <pre>
- * final DateFormatSupplier dfSupplier = DateFormatSupplier.createSimpleFormatSupplier(
- *     DateFormatSupplier.HUMAN_READABLE_FORMAT, TimeZone.getTimeZone(&quot;UTC&quot;));
- * final FormatterConfig config =
- *     new FormatterConfig().setPrintTimestamps(true).setDateFormatSupplier(dfSupplier);
- * </pre>
- *
- * @deprecated since 1.8.0
- */
-@Deprecated(since = "1.8.0")
-public class DateStringFormatter implements Formatter {
-
-  private DefaultFormatter defaultFormatter;
-  private TimeZone timeZone;
-
-  public static final String DATE_FORMAT = DateFormatSupplier.HUMAN_READABLE_FORMAT;
-
-  public DateStringFormatter() {
-    this(TimeZone.getDefault());
-  }
-
-  public DateStringFormatter(TimeZone timeZone) {
-    this.defaultFormatter = new DefaultFormatter();
-    this.timeZone = timeZone;
-  }
-
-  @Override
-  public void initialize(Iterable<Entry<Key,Value>> scanner, FormatterConfig config) {
-    FormatterConfig newConfig = new FormatterConfig(config);
-    newConfig.setDateFormatSupplier(
-        DateFormatSupplier.createSimpleFormatSupplier(DATE_FORMAT, timeZone));
-    defaultFormatter.initialize(scanner, newConfig);
-  }
-
-  @Override
-  public boolean hasNext() {
-    return defaultFormatter.hasNext();
-  }
-
-  @Override
-  public String next() {
-    return defaultFormatter.next();
-  }
-
-  @Override
-  public void remove() {
-    defaultFormatter.remove();
-  }
-
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/util/format/HexFormatter.java b/core/src/main/java/org/apache/accumulo/core/util/format/HexFormatter.java
deleted file mode 100644
index 1c7e07c..0000000
--- a/core/src/main/java/org/apache/accumulo/core/util/format/HexFormatter.java
+++ /dev/null
@@ -1,141 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.util.format;
-
-import java.util.Iterator;
-import java.util.Map.Entry;
-
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.util.interpret.ScanInterpreter;
-import org.apache.hadoop.io.Text;
-
-/**
- * A simple formatter that print the row, column family, column qualifier, and value as hex
- */
-@Deprecated(since = "2.1.0")
-public class HexFormatter implements Formatter, ScanInterpreter {
-
-  private char[] chars =
-      {'0', '1', '2', '3', '4', '5', '6', '7', '8', '9', 'a', 'b', 'c', 'd', 'e', 'f'};
-  private Iterator<Entry<Key,Value>> iter;
-  private FormatterConfig config;
-
-  private void toHex(StringBuilder sb, byte[] bin) {
-
-    for (int i = 0; i < bin.length; i++) {
-      if (i > 0 && i % 2 == 0) {
-        sb.append('-');
-      }
-      sb.append(chars[0x0f & (bin[i] >>> 4)]);
-      sb.append(chars[0x0f & bin[i]]);
-    }
-  }
-
-  private int fromChar(char b) {
-    if (b >= '0' && b <= '9') {
-      return (b - '0');
-    } else if (b >= 'a' && b <= 'f') {
-      return (b - 'a' + 10);
-    }
-
-    throw new IllegalArgumentException("Bad char " + b);
-  }
-
-  private byte[] toBinary(String hex) {
-    hex = hex.replace("-", "");
-
-    byte[] bin = new byte[(hex.length() / 2) + (hex.length() % 2)];
-
-    int j = 0;
-    for (int i = 0; i < bin.length; i++) {
-      bin[i] = (byte) (fromChar(hex.charAt(j++)) << 4);
-      if (j >= hex.length()) {
-        break;
-      }
-      bin[i] |= (byte) fromChar(hex.charAt(j++));
-    }
-
-    return bin;
-  }
-
-  @Override
-  public boolean hasNext() {
-    return iter.hasNext();
-  }
-
-  @Override
-  public String next() {
-    Entry<Key,Value> entry = iter.next();
-
-    StringBuilder sb = new StringBuilder();
-
-    toHex(sb, entry.getKey().getRowData().toArray());
-    sb.append("  ");
-    toHex(sb, entry.getKey().getColumnFamilyData().toArray());
-    sb.append("  ");
-    toHex(sb, entry.getKey().getColumnQualifierData().toArray());
-    sb.append(" [");
-    sb.append(entry.getKey().getColumnVisibilityData());
-    sb.append("] ");
-    if (config.willPrintTimestamps()) {
-      sb.append(entry.getKey().getTimestamp());
-      sb.append("  ");
-    }
-    toHex(sb, entry.getValue().get());
-
-    return sb.toString();
-  }
-
-  @Override
-  public void remove() {
-    iter.remove();
-  }
-
-  @Override
-  public void initialize(Iterable<Entry<Key,Value>> scanner, FormatterConfig config) {
-    this.iter = scanner.iterator();
-    this.config = new FormatterConfig(config);
-  }
-
-  @Override
-  public Text interpretRow(Text row) {
-    return new Text(toBinary(row.toString()));
-  }
-
-  @Override
-  public Text interpretBeginRow(Text row) {
-    return interpretRow(row);
-  }
-
-  @Override
-  public Text interpretEndRow(Text row) {
-    return interpretRow(row);
-  }
-
-  @Override
-  public Text interpretColumnFamily(Text cf) {
-    return interpretRow(cf);
-  }
-
-  @Override
-  public Text interpretColumnQualifier(Text cq) {
-    return interpretRow(cq);
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/util/interpret/DefaultScanInterpreter.java b/core/src/main/java/org/apache/accumulo/core/util/interpret/DefaultScanInterpreter.java
deleted file mode 100644
index 25086ff..0000000
--- a/core/src/main/java/org/apache/accumulo/core/util/interpret/DefaultScanInterpreter.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.util.interpret;
-
-import org.apache.hadoop.io.Text;
-
-/**
- * @deprecated since 2.1.0 This will be removed in a future version in favor of JShell
- */
-@Deprecated(since = "2.1.0")
-public class DefaultScanInterpreter implements ScanInterpreter {
-
-  @Override
-  public Text interpretRow(Text row) {
-    return row;
-  }
-
-  @Override
-  public Text interpretBeginRow(Text row) {
-    return row;
-  }
-
-  @Override
-  public Text interpretEndRow(Text row) {
-    return row;
-  }
-
-  @Override
-  public Text interpretColumnFamily(Text cf) {
-    return cf;
-  }
-
-  @Override
-  public Text interpretColumnQualifier(Text cq) {
-    return cq;
-  }
-
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/util/interpret/HexScanInterpreter.java b/core/src/main/java/org/apache/accumulo/core/util/interpret/HexScanInterpreter.java
deleted file mode 100644
index 09a0400..0000000
--- a/core/src/main/java/org/apache/accumulo/core/util/interpret/HexScanInterpreter.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.util.interpret;
-
-import org.apache.accumulo.core.util.format.HexFormatter;
-
-/**
- * As simple scan interpreter that converts hex to binary. IT supports translating the output of
- * {@link HexFormatter} back to binary. The hex input can contain dashes (because
- * {@link HexFormatter} outputs dashes) which are ignored.
- */
-@Deprecated(since = "2.1.0")
-public class HexScanInterpreter extends HexFormatter {
-
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/util/interpret/ScanInterpreter.java b/core/src/main/java/org/apache/accumulo/core/util/interpret/ScanInterpreter.java
deleted file mode 100644
index 7cac40a..0000000
--- a/core/src/main/java/org/apache/accumulo/core/util/interpret/ScanInterpreter.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.util.interpret;
-
-import org.apache.hadoop.io.Text;
-
-/**
- * A simple interface for creating shell plugins that translate the range and column arguments for
- * the shell's scan command.
- *
- * @deprecated since 2.1.0 This will be removed in a future version in favor of JShell
- */
-@Deprecated(since = "2.1.0")
-public interface ScanInterpreter {
-
-  Text interpretRow(Text row);
-
-  Text interpretBeginRow(Text row);
-
-  Text interpretEndRow(Text row);
-
-  Text interpretColumnFamily(Text cf);
-
-  Text interpretColumnQualifier(Text cq);
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/util/json/ByteArrayToBase64TypeAdapter.java b/core/src/main/java/org/apache/accumulo/core/util/json/ByteArrayToBase64TypeAdapter.java
new file mode 100644
index 0000000..2499d78
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/util/json/ByteArrayToBase64TypeAdapter.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.util.json;
+
+import java.lang.reflect.Type;
+import java.util.Base64;
+import java.util.Base64.Decoder;
+import java.util.Base64.Encoder;
+import java.util.Objects;
+
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import com.google.gson.JsonDeserializationContext;
+import com.google.gson.JsonDeserializer;
+import com.google.gson.JsonElement;
+import com.google.gson.JsonParseException;
+import com.google.gson.JsonPrimitive;
+import com.google.gson.JsonSerializationContext;
+import com.google.gson.JsonSerializer;
+
+/**
+ * Gson adapter to handle serializing and deserializing byte arrays using Base64 encoding.
+ */
+public class ByteArrayToBase64TypeAdapter
+    implements JsonSerializer<byte[]>, JsonDeserializer<byte[]> {
+
+  private static final Decoder decoder = Base64.getUrlDecoder();
+  private static final Encoder encoder = Base64.getUrlEncoder();
+
+  @Override
+  public byte[] deserialize(JsonElement json, Type typeOfT, JsonDeserializationContext context)
+      throws JsonParseException {
+    return decoder.decode(json.getAsString());
+  }
+
+  @Override
+  public JsonElement serialize(byte[] src, Type typeOfSrc, JsonSerializationContext context) {
+    return new JsonPrimitive(encoder.encodeToString(src));
+  }
+
+  /**
+   * Creates a new Gson instance that registers {@link ByteArrayToBase64TypeAdapter} for handling
+   * serializing/deserializing byte[] types as Base64 encoded
+   *
+   * @return Gson instance
+   */
+  public static Gson createBase64Gson() {
+    return registerBase64TypeAdapter(new GsonBuilder()).create();
+  }
+
+  /**
+   * Register {@link ByteArrayToBase64TypeAdapter} for handling byte[] types on an existing
+   * GsonBuilder
+   *
+   * @param gsonBuilder existing GsonBuilder
+   * @return GsonBuilder
+   */
+  public static GsonBuilder registerBase64TypeAdapter(final GsonBuilder gsonBuilder) {
+    return Objects.requireNonNull(gsonBuilder).disableHtmlEscaping()
+        .registerTypeHierarchyAdapter(byte[].class, new ByteArrayToBase64TypeAdapter());
+  }
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/util/tables/TableZooHelper.java b/core/src/main/java/org/apache/accumulo/core/util/tables/TableZooHelper.java
index de53d2b..7d8cb85 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/tables/TableZooHelper.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/tables/TableZooHelper.java
@@ -25,7 +25,6 @@
 
 import java.util.List;
 import java.util.Objects;
-import java.util.concurrent.ExecutionException;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.NamespaceNotFoundException;
@@ -37,11 +36,10 @@
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.fate.zookeeper.ZooCache;
 import org.apache.accumulo.core.manager.state.tables.TableState;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 
-import com.google.common.cache.Cache;
-import com.google.common.cache.CacheBuilder;
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
 
 public class TableZooHelper implements AutoCloseable {
 
@@ -49,7 +47,7 @@
   // Per instance cache will expire after 10 minutes in case we
   // encounter an instance not used frequently
   private final Cache<TableZooHelper,TableMap> instanceToMapCache =
-      CacheBuilder.newBuilder().expireAfterAccess(10, MINUTES).build();
+      Caffeine.newBuilder().expireAfterAccess(10, MINUTES).build();
 
   public TableZooHelper(ClientContext context) {
     this.context = Objects.requireNonNull(context);
@@ -62,11 +60,10 @@
    *         getCause() of NamespaceNotFoundException
    */
   public TableId getTableId(String tableName) throws TableNotFoundException {
-    if (MetadataTable.NAME.equals(tableName)) {
-      return MetadataTable.ID;
-    }
-    if (RootTable.NAME.equals(tableName)) {
-      return RootTable.ID;
+    for (AccumuloTable systemTable : AccumuloTable.values()) {
+      if (systemTable.tableName().equals(tableName)) {
+        return systemTable.tableId();
+      }
     }
     try {
       return _getTableIdDetectNamespaceNotFound(EXISTING_TABLE_NAME.validate(tableName));
@@ -99,11 +96,10 @@
   }
 
   public String getTableName(TableId tableId) throws TableNotFoundException {
-    if (MetadataTable.ID.equals(tableId)) {
-      return MetadataTable.NAME;
-    }
-    if (RootTable.ID.equals(tableId)) {
-      return RootTable.NAME;
+    for (AccumuloTable systemTable : AccumuloTable.values()) {
+      if (systemTable.tableId().equals(tableId)) {
+        return systemTable.tableName();
+      }
     }
     String tableName = getTableMap().getIdtoNameMap().get(tableId);
     if (tableName == null) {
@@ -127,11 +123,7 @@
   }
 
   private TableMap getCachedTableMap() {
-    try {
-      return instanceToMapCache.get(this, () -> new TableMap(context));
-    } catch (ExecutionException e) {
-      throw new RuntimeException(e);
-    }
+    return instanceToMapCache.get(this, k -> new TableMap(context));
   }
 
   public boolean tableNodeExists(TableId tableId) {
@@ -201,7 +193,7 @@
     checkArgument(context != null, "instance is null");
     checkArgument(tableId != null, "tableId is null");
 
-    if (MetadataTable.ID.equals(tableId) || RootTable.ID.equals(tableId)) {
+    if (AccumuloTable.allTableIds().contains(tableId)) {
       return Namespace.ACCUMULO.id();
     }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/util/threads/ThreadPools.java b/core/src/main/java/org/apache/accumulo/core/util/threads/ThreadPools.java
index ea26c56..324fafe 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/threads/ThreadPools.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/threads/ThreadPools.java
@@ -91,27 +91,24 @@
   private static final ConcurrentLinkedQueue<ScheduledFuture<?>> NON_CRITICAL_RUNNING_TASKS =
       new ConcurrentLinkedQueue<>();
 
-  private static final Runnable TASK_CHECKER = new Runnable() {
-    @Override
-    public void run() {
-      final List<ConcurrentLinkedQueue<ScheduledFuture<?>>> queues =
-          List.of(CRITICAL_RUNNING_TASKS, NON_CRITICAL_RUNNING_TASKS);
-      while (true) {
-        queues.forEach(q -> {
-          Iterator<ScheduledFuture<?>> tasks = q.iterator();
-          while (tasks.hasNext()) {
-            if (checkTaskFailed(tasks.next(), q)) {
-              tasks.remove();
-            }
+  private static Runnable TASK_CHECKER = () -> {
+    final List<ConcurrentLinkedQueue<ScheduledFuture<?>>> queues =
+        List.of(CRITICAL_RUNNING_TASKS, NON_CRITICAL_RUNNING_TASKS);
+    while (true) {
+      queues.forEach(q -> {
+        Iterator<ScheduledFuture<?>> tasks = q.iterator();
+        while (tasks.hasNext()) {
+          if (checkTaskFailed(tasks.next(), q)) {
+            tasks.remove();
           }
-        });
-        try {
-          MINUTES.sleep(1);
-        } catch (InterruptedException ie) {
-          // This thread was interrupted by something while sleeping. We don't want to exit
-          // this thread, so reset the interrupt state on this thread and keep going.
-          Thread.interrupted();
         }
+      });
+      try {
+        TimeUnit.MINUTES.sleep(1);
+      } catch (InterruptedException ie) {
+        // This thread was interrupted by something while sleeping. We don't want to exit
+        // this thread, so reset the interrupt state on this thread and keep going.
+        Thread.interrupted();
       }
     }
   };
@@ -266,29 +263,13 @@
    * @return ExecutorService impl
    * @throws IllegalArgumentException if property is not handled
    */
-  @SuppressWarnings("deprecation")
   public ThreadPoolExecutor createExecutorService(final AccumuloConfiguration conf,
       final Property p, boolean emitThreadPoolMetrics) {
     ThreadPoolExecutorBuilder builder;
     switch (p) {
-      case GENERAL_SIMPLETIMER_THREADPOOL_SIZE:
-        return createScheduledExecutorService(conf.getCount(p), "SimpleTimer");
       case GENERAL_THREADPOOL_SIZE:
         return createScheduledExecutorService(conf.getCount(p), "GeneralExecutor",
             emitThreadPoolMetrics);
-      case MANAGER_BULK_THREADPOOL_SIZE:
-        builder = getPoolBuilder("bulk import").numCoreThreads(conf.getCount(p)).withTimeOut(
-            conf.getTimeInMillis(Property.MANAGER_BULK_THREADPOOL_TIMEOUT), MILLISECONDS);
-        if (emitThreadPoolMetrics) {
-          builder.enableThreadPoolMetrics();
-        }
-        return builder.build();
-      case MANAGER_RENAME_THREADS:
-        builder = getPoolBuilder("bulk move").numCoreThreads(conf.getCount(p));
-        if (emitThreadPoolMetrics) {
-          builder.enableThreadPoolMetrics();
-        }
-        return builder.build();
       case MANAGER_FATE_THREADPOOL_SIZE:
         builder = getPoolBuilder("Repo Runner").numCoreThreads(conf.getCount(p));
         if (emitThreadPoolMetrics) {
@@ -358,13 +339,6 @@
         return builder.build();
       case GC_DELETE_THREADS:
         return getPoolBuilder("deleting").numCoreThreads(conf.getCount(p)).build();
-      case REPLICATION_WORKER_THREADS:
-        builder = getPoolBuilder("replication task").numCoreThreads(conf.getCount(p));
-        if (emitThreadPoolMetrics) {
-          builder.enableThreadPoolMetrics();
-        }
-        return builder.build();
-
       default:
         throw new IllegalArgumentException("Unhandled thread pool property: " + p);
     }
@@ -566,10 +540,8 @@
    */
   public ScheduledThreadPoolExecutor
       createGeneralScheduledExecutorService(AccumuloConfiguration conf) {
-    @SuppressWarnings("deprecation")
-    Property oldProp = Property.GENERAL_SIMPLETIMER_THREADPOOL_SIZE;
-    Property prop = conf.resolve(Property.GENERAL_THREADPOOL_SIZE, oldProp);
-    return (ScheduledThreadPoolExecutor) createExecutorService(conf, prop, true);
+    return (ScheduledThreadPoolExecutor) createExecutorService(conf,
+        Property.GENERAL_THREADPOOL_SIZE, true);
   }
 
   /**
diff --git a/core/src/main/java/org/apache/accumulo/core/util/time/NanoTime.java b/core/src/main/java/org/apache/accumulo/core/util/time/NanoTime.java
new file mode 100644
index 0000000..f081278
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/util/time/NanoTime.java
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.util.time;
+
+import java.time.Duration;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * This class implements a strong type for System.nanoTime() that offers the limited operations that
+ * can be performed on a nanoTime. See the System.nanoTime() javadoc for details - specifically
+ * these values are meaningful only when the difference between two such values, obtained within the
+ * same instance of a Java virtual machine, are computed.
+ */
+public final class NanoTime implements Comparable<NanoTime> {
+  // In the System.nanoTime javadoc it describes the returned value as the "nanoseconds since some
+  // fixed but arbitrary origin time (perhaps in the future, so values may be negative)". This
+  // variable name is derived from that where AO is arbitrary origin.
+  private final long nanosSinceAO;
+
+  // This method should only be called by test inorder to test edge conditions, that is why it is
+  // package private. Calling this outside of test makes it hard to reason about the correctness of
+  // using this class.
+  @VisibleForTesting
+  NanoTime(long ntsao) {
+    this.nanosSinceAO = ntsao;
+  }
+
+  /**
+   * @return this.nanoTime - other.nanoTime as a Duration
+   */
+  public Duration subtract(NanoTime other) {
+    return Duration.ofNanos(nanosSinceAO - other.nanosSinceAO);
+  }
+
+  /**
+   * Determines the amount of time that has elapsed since this object was created relative to the
+   * current nanoTime.
+   *
+   * @return System.nanoTime() - this.nanoTime
+   */
+  public Duration elapsed() {
+    return Duration.ofNanos(System.nanoTime() - nanosSinceAO);
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other instanceof NanoTime) {
+      return nanosSinceAO == ((NanoTime) other).nanosSinceAO;
+    }
+
+    return false;
+  }
+
+  @Override
+  public int hashCode() {
+    return Long.hashCode(nanosSinceAO);
+  }
+
+  @Override
+  public int compareTo(NanoTime other) {
+    // All operations w/ nanoTimes must use differences, can not directly compare. This is because a
+    // nano time of Long.MAX_VALUE -10 is considered less than Long.MAX_VALUE +10
+    long diff = nanosSinceAO - other.nanosSinceAO;
+
+    if (diff < 0) {
+      return -1;
+    } else if (diff > 0) {
+      return 1;
+    } else {
+      return 0;
+    }
+  }
+
+  /**
+   * @return a NanoTime created using System.nanoTime()
+   */
+  public static NanoTime now() {
+    return new NanoTime(System.nanoTime());
+  }
+
+  /**
+   * @return a NanoTime created using System.nanoTime() + duration.toNanos()
+   */
+  public static NanoTime nowPlus(Duration duration) {
+    return new NanoTime(System.nanoTime() + duration.toNanos());
+  }
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/util/time/SteadyTime.java b/core/src/main/java/org/apache/accumulo/core/util/time/SteadyTime.java
new file mode 100644
index 0000000..d16f15c
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/util/time/SteadyTime.java
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.util.time;
+
+import java.time.Duration;
+import java.util.Objects;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * SteadyTime represents an approximation of the total duration of time this cluster has had a
+ * Manager. Because this represents an elapsed time it is guaranteed to not be negative. SteadyTime
+ * is not expected to represent real world date times, its main use is for computing deltas similar
+ * System.nanoTime but across JVM processes.
+ */
+public class SteadyTime implements Comparable<SteadyTime> {
+
+  private final Duration time;
+
+  private SteadyTime(Duration time) {
+    Preconditions.checkArgument(!time.isNegative(), "SteadyTime '%s' should not be negative.",
+        time.toNanos());
+    this.time = time;
+  }
+
+  public long getMillis() {
+    return time.toMillis();
+  }
+
+  public long getNanos() {
+    return time.toNanos();
+  }
+
+  public Duration getDuration() {
+    return time;
+  }
+
+  public Duration minus(SteadyTime other) {
+    return time.minus(other.getDuration());
+  }
+
+  public SteadyTime plus(Duration other) {
+    return SteadyTime.from(time.plus(other));
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    SteadyTime that = (SteadyTime) o;
+    return Objects.equals(time, that.time);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hashCode(time);
+  }
+
+  @Override
+  public int compareTo(SteadyTime other) {
+    return time.compareTo(other.time);
+  }
+
+  @Override
+  public String toString() {
+    return "SteadyTime[" + time + "]";
+  }
+
+  public static SteadyTime from(long time, TimeUnit unit) {
+    return new SteadyTime(Duration.of(time, unit.toChronoUnit()));
+  }
+
+  public static SteadyTime from(Duration time) {
+    return new SteadyTime(time);
+  }
+
+}
diff --git a/core/src/main/scripts/generate-thrift.sh b/core/src/main/scripts/generate-thrift.sh
index 37929ee..0ad5911 100755
--- a/core/src/main/scripts/generate-thrift.sh
+++ b/core/src/main/scripts/generate-thrift.sh
@@ -32,7 +32,7 @@
 [[ -z $REQUIRED_THRIFT_VERSION ]] && REQUIRED_THRIFT_VERSION='0.17.0'
 [[ -z $INCLUDED_MODULES ]] && INCLUDED_MODULES=()
 [[ -z $BASE_OUTPUT_PACKAGE ]] && BASE_OUTPUT_PACKAGE='org.apache.accumulo.core'
-[[ -z $PACKAGES_TO_GENERATE ]] && PACKAGES_TO_GENERATE=(gc master manager tabletserver securityImpl clientImpl dataImpl replication trace compaction)
+[[ -z $PACKAGES_TO_GENERATE ]] && PACKAGES_TO_GENERATE=(gc manager tabletserver securityImpl clientImpl dataImpl compaction tabletingest tablet tabletscan)
 [[ -z $BUILD_DIR ]] && BUILD_DIR='target'
 [[ -z $LANGUAGES_TO_GENERATE ]] && LANGUAGES_TO_GENERATE=(java)
 [[ -z $FINAL_DIR ]] && FINAL_DIR='src/main'
diff --git a/core/src/main/spotbugs/exclude-filter.xml b/core/src/main/spotbugs/exclude-filter.xml
index cc0bd6a..ded70a9 100644
--- a/core/src/main/spotbugs/exclude-filter.xml
+++ b/core/src/main/spotbugs/exclude-filter.xml
@@ -22,6 +22,7 @@
   <!--
     DO NOT exclude anything other than generated files here. Other files
     can be excluded inline by adding the @SuppressFBWarnings annotation.
+    Exceptions can be made if the bug is particularly spammy or trivial.
   -->
   <Match>
     <!-- ignore thrift-generated files -->
@@ -31,9 +32,10 @@
       <Package name="org.apache.accumulo.core.dataImpl.thrift" />
       <Package name="org.apache.accumulo.core.gc.thrift" />
       <Package name="org.apache.accumulo.core.manager.thrift" />
-      <Package name="org.apache.accumulo.core.master.thrift" />
-      <Package name="org.apache.accumulo.core.replication.thrift" />
       <Package name="org.apache.accumulo.core.securityImpl.thrift" />
+      <Package name="org.apache.accumulo.core.tablet.thrift" />
+      <Package name="org.apache.accumulo.core.tabletingest.thrift" />
+      <Package name="org.apache.accumulo.core.tabletscan.thrift" />
       <Package name="org.apache.accumulo.core.tabletserver.thrift" />
       <Package name="org.apache.accumulo.core.trace.thrift" />
     </Or>
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 91b78f9..64713af 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
@@ -35,71 +35,69 @@
 
     public java.lang.String getZooKeepers() throws org.apache.thrift.TException;
 
-    public java.util.List<java.lang.String> bulkImportFiles(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long tid, java.lang.String tableId, java.util.List<java.lang.String> files, java.lang.String errorDir, boolean setTime) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException;
-
-    public boolean isActive(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long tid) throws org.apache.thrift.TException;
+    public boolean isActive(TInfo tinfo, long tid) throws org.apache.thrift.TException;
 
     public void ping(org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws ThriftSecurityException, org.apache.thrift.TException;
 
     public java.util.List<TDiskUsage> getDiskUsage(java.util.Set<java.lang.String> tables, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException;
 
-    public java.util.Set<java.lang.String> listLocalUsers(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws ThriftSecurityException, org.apache.thrift.TException;
+    public java.util.Set<java.lang.String> listLocalUsers(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws ThriftSecurityException, org.apache.thrift.TException;
 
-    public void createLocalUser(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.nio.ByteBuffer password) throws ThriftSecurityException, org.apache.thrift.TException;
+    public void createLocalUser(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.nio.ByteBuffer password) throws ThriftSecurityException, org.apache.thrift.TException;
 
-    public void dropLocalUser(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal) throws ThriftSecurityException, org.apache.thrift.TException;
+    public void dropLocalUser(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal) throws ThriftSecurityException, org.apache.thrift.TException;
 
-    public void changeLocalUserPassword(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.nio.ByteBuffer password) throws ThriftSecurityException, org.apache.thrift.TException;
+    public void changeLocalUserPassword(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.nio.ByteBuffer password) throws ThriftSecurityException, org.apache.thrift.TException;
 
-    public boolean authenticate(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws ThriftSecurityException, org.apache.thrift.TException;
+    public boolean authenticate(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws ThriftSecurityException, org.apache.thrift.TException;
 
-    public boolean authenticateUser(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.securityImpl.thrift.TCredentials toAuth) throws ThriftSecurityException, org.apache.thrift.TException;
+    public boolean authenticateUser(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.securityImpl.thrift.TCredentials toAuth) throws ThriftSecurityException, org.apache.thrift.TException;
 
-    public void changeAuthorizations(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.util.List<java.nio.ByteBuffer> authorizations) throws ThriftSecurityException, org.apache.thrift.TException;
+    public void changeAuthorizations(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.util.List<java.nio.ByteBuffer> authorizations) throws ThriftSecurityException, org.apache.thrift.TException;
 
-    public java.util.List<java.nio.ByteBuffer> getUserAuthorizations(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal) throws ThriftSecurityException, org.apache.thrift.TException;
+    public java.util.List<java.nio.ByteBuffer> getUserAuthorizations(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal) throws ThriftSecurityException, org.apache.thrift.TException;
 
-    public boolean hasSystemPermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte sysPerm) throws ThriftSecurityException, org.apache.thrift.TException;
+    public boolean hasSystemPermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte sysPerm) throws ThriftSecurityException, org.apache.thrift.TException;
 
-    public boolean hasTablePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte tblPerm) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException;
+    public boolean hasTablePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte tblPerm) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException;
 
-    public boolean hasNamespacePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte tblNspcPerm) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException;
+    public boolean hasNamespacePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte tblNspcPerm) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException;
 
-    public void grantSystemPermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte permission) throws ThriftSecurityException, org.apache.thrift.TException;
+    public void grantSystemPermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte permission) throws ThriftSecurityException, org.apache.thrift.TException;
 
-    public void revokeSystemPermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte permission) throws ThriftSecurityException, org.apache.thrift.TException;
+    public void revokeSystemPermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte permission) throws ThriftSecurityException, org.apache.thrift.TException;
 
-    public void grantTablePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte permission) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException;
+    public void grantTablePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte permission) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException;
 
-    public void revokeTablePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte permission) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException;
+    public void revokeTablePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte permission) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException;
 
-    public void grantNamespacePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte permission) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException;
+    public void grantNamespacePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte permission) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException;
 
-    public void revokeNamespacePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte permission) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException;
+    public void revokeNamespacePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte permission) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException;
 
-    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 ThriftSecurityException, org.apache.thrift.TException;
+    public java.util.Map<java.lang.String,java.lang.String> getConfiguration(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, ConfigurationType type) throws ThriftSecurityException, 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 ThriftSecurityException, org.apache.thrift.TException;
+    public java.util.Map<java.lang.String,java.lang.String> getSystemProperties(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws ThriftSecurityException, org.apache.thrift.TException;
 
-    public TVersionedProperties getVersionedSystemProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws ThriftSecurityException, org.apache.thrift.TException;
+    public TVersionedProperties getVersionedSystemProperties(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws ThriftSecurityException, 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, ThriftSecurityException, org.apache.thrift.TException;
+    public java.util.Map<java.lang.String,java.lang.String> getTableConfiguration(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName) throws ThriftTableOperationException, ThriftSecurityException, 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, ThriftSecurityException, org.apache.thrift.TException;
+    public java.util.Map<java.lang.String,java.lang.String> getTableProperties(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName) throws ThriftTableOperationException, ThriftSecurityException, 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, ThriftSecurityException, org.apache.thrift.TException;
+    public TVersionedProperties getVersionedTableProperties(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName) throws ThriftTableOperationException, ThriftSecurityException, 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, ThriftSecurityException, org.apache.thrift.TException;
+    public java.util.Map<java.lang.String,java.lang.String> getNamespaceConfiguration(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns) throws ThriftTableOperationException, ThriftSecurityException, 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, ThriftSecurityException, org.apache.thrift.TException;
+    public java.util.Map<java.lang.String,java.lang.String> getNamespaceProperties(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns) throws ThriftTableOperationException, ThriftSecurityException, 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, ThriftSecurityException, org.apache.thrift.TException;
+    public TVersionedProperties getVersionedNamespaceProperties(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns) throws ThriftTableOperationException, ThriftSecurityException, 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 checkClass(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;
+    public boolean checkTableClass(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;
 
-    public boolean checkNamespaceClass(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String namespaceId, java.lang.String className, java.lang.String interfaceMatch) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException;
+    public boolean checkNamespaceClass(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String namespaceId, java.lang.String className, java.lang.String interfaceMatch) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException;
 
   }
 
@@ -111,71 +109,69 @@
 
     public void getZooKeepers(org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler) throws org.apache.thrift.TException;
 
-    public void bulkImportFiles(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long tid, java.lang.String tableId, java.util.List<java.lang.String> files, java.lang.String errorDir, boolean setTime, org.apache.thrift.async.AsyncMethodCallback<java.util.List<java.lang.String>> resultHandler) throws org.apache.thrift.TException;
-
-    public void isActive(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long tid, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException;
+    public void isActive(TInfo tinfo, long tid, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException;
 
     public void ping(org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
 
     public void getDiskUsage(java.util.Set<java.lang.String> tables, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.List<TDiskUsage>> resultHandler) throws org.apache.thrift.TException;
 
-    public void listLocalUsers(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.Set<java.lang.String>> resultHandler) throws org.apache.thrift.TException;
+    public void listLocalUsers(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.Set<java.lang.String>> resultHandler) throws org.apache.thrift.TException;
 
-    public void createLocalUser(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.nio.ByteBuffer password, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+    public void createLocalUser(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.nio.ByteBuffer password, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
 
-    public void dropLocalUser(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+    public void dropLocalUser(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
 
-    public void changeLocalUserPassword(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.nio.ByteBuffer password, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+    public void changeLocalUserPassword(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.nio.ByteBuffer password, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
 
-    public void authenticate(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException;
+    public void authenticate(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException;
 
-    public void authenticateUser(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.securityImpl.thrift.TCredentials toAuth, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException;
+    public void authenticateUser(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.securityImpl.thrift.TCredentials toAuth, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException;
 
-    public void changeAuthorizations(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.util.List<java.nio.ByteBuffer> authorizations, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+    public void changeAuthorizations(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.util.List<java.nio.ByteBuffer> authorizations, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
 
-    public void getUserAuthorizations(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, org.apache.thrift.async.AsyncMethodCallback<java.util.List<java.nio.ByteBuffer>> resultHandler) throws org.apache.thrift.TException;
+    public void getUserAuthorizations(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, org.apache.thrift.async.AsyncMethodCallback<java.util.List<java.nio.ByteBuffer>> resultHandler) throws org.apache.thrift.TException;
 
-    public void hasSystemPermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte sysPerm, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException;
+    public void hasSystemPermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte sysPerm, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException;
 
-    public void hasTablePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte tblPerm, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException;
+    public void hasTablePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte tblPerm, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException;
 
-    public void hasNamespacePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte tblNspcPerm, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException;
+    public void hasNamespacePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte tblNspcPerm, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException;
 
-    public void grantSystemPermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+    public void grantSystemPermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
 
-    public void revokeSystemPermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+    public void revokeSystemPermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
 
-    public void grantTablePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+    public void grantTablePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
 
-    public void revokeTablePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+    public void revokeTablePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
 
-    public void grantNamespacePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+    public void grantNamespacePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
 
-    public void revokeNamespacePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+    public void revokeNamespacePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
 
-    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 getConfiguration(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 getSystemProperties(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 getVersionedSystemProperties(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 getTableConfiguration(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 getTableProperties(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 getVersionedTableProperties(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 getNamespaceConfiguration(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 getNamespaceProperties(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 getVersionedNamespaceProperties(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 checkClass(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;
+    public void checkTableClass(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;
 
-    public void checkNamespaceClass(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String namespaceId, java.lang.String className, java.lang.String interfaceMatch, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException;
+    public void checkNamespaceClass(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String namespaceId, java.lang.String className, java.lang.String interfaceMatch, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException;
 
   }
 
@@ -271,49 +267,13 @@
     }
 
     @Override
-    public java.util.List<java.lang.String> bulkImportFiles(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long tid, java.lang.String tableId, java.util.List<java.lang.String> files, java.lang.String errorDir, boolean setTime) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
-    {
-      send_bulkImportFiles(tinfo, credentials, tid, tableId, files, errorDir, setTime);
-      return recv_bulkImportFiles();
-    }
-
-    public void send_bulkImportFiles(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long tid, java.lang.String tableId, java.util.List<java.lang.String> files, java.lang.String errorDir, boolean setTime) throws org.apache.thrift.TException
-    {
-      bulkImportFiles_args args = new bulkImportFiles_args();
-      args.setTinfo(tinfo);
-      args.setCredentials(credentials);
-      args.setTid(tid);
-      args.setTableId(tableId);
-      args.setFiles(files);
-      args.setErrorDir(errorDir);
-      args.setSetTime(setTime);
-      sendBase("bulkImportFiles", args);
-    }
-
-    public java.util.List<java.lang.String> recv_bulkImportFiles() throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
-    {
-      bulkImportFiles_result result = new bulkImportFiles_result();
-      receiveBase(result, "bulkImportFiles");
-      if (result.isSetSuccess()) {
-        return result.success;
-      }
-      if (result.sec != null) {
-        throw result.sec;
-      }
-      if (result.tope != null) {
-        throw result.tope;
-      }
-      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "bulkImportFiles failed: unknown result");
-    }
-
-    @Override
-    public boolean isActive(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long tid) throws org.apache.thrift.TException
+    public boolean isActive(TInfo tinfo, long tid) throws org.apache.thrift.TException
     {
       send_isActive(tinfo, tid);
       return recv_isActive();
     }
 
-    public void send_isActive(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long tid) throws org.apache.thrift.TException
+    public void send_isActive(TInfo tinfo, long tid) throws org.apache.thrift.TException
     {
       isActive_args args = new isActive_args();
       args.setTinfo(tinfo);
@@ -387,13 +347,13 @@
     }
 
     @Override
-    public java.util.Set<java.lang.String> listLocalUsers(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws ThriftSecurityException, org.apache.thrift.TException
+    public java.util.Set<java.lang.String> listLocalUsers(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws ThriftSecurityException, org.apache.thrift.TException
     {
       send_listLocalUsers(tinfo, credentials);
       return recv_listLocalUsers();
     }
 
-    public void send_listLocalUsers(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
+    public void send_listLocalUsers(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
     {
       listLocalUsers_args args = new listLocalUsers_args();
       args.setTinfo(tinfo);
@@ -415,13 +375,13 @@
     }
 
     @Override
-    public void createLocalUser(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.nio.ByteBuffer password) throws ThriftSecurityException, org.apache.thrift.TException
+    public void createLocalUser(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.nio.ByteBuffer password) throws ThriftSecurityException, org.apache.thrift.TException
     {
       send_createLocalUser(tinfo, credentials, principal, password);
       recv_createLocalUser();
     }
 
-    public void send_createLocalUser(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.nio.ByteBuffer password) throws org.apache.thrift.TException
+    public void send_createLocalUser(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.nio.ByteBuffer password) throws org.apache.thrift.TException
     {
       createLocalUser_args args = new createLocalUser_args();
       args.setTinfo(tinfo);
@@ -442,13 +402,13 @@
     }
 
     @Override
-    public void dropLocalUser(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal) throws ThriftSecurityException, org.apache.thrift.TException
+    public void dropLocalUser(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal) throws ThriftSecurityException, org.apache.thrift.TException
     {
       send_dropLocalUser(tinfo, credentials, principal);
       recv_dropLocalUser();
     }
 
-    public void send_dropLocalUser(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal) throws org.apache.thrift.TException
+    public void send_dropLocalUser(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal) throws org.apache.thrift.TException
     {
       dropLocalUser_args args = new dropLocalUser_args();
       args.setTinfo(tinfo);
@@ -468,13 +428,13 @@
     }
 
     @Override
-    public void changeLocalUserPassword(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.nio.ByteBuffer password) throws ThriftSecurityException, org.apache.thrift.TException
+    public void changeLocalUserPassword(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.nio.ByteBuffer password) throws ThriftSecurityException, org.apache.thrift.TException
     {
       send_changeLocalUserPassword(tinfo, credentials, principal, password);
       recv_changeLocalUserPassword();
     }
 
-    public void send_changeLocalUserPassword(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.nio.ByteBuffer password) throws org.apache.thrift.TException
+    public void send_changeLocalUserPassword(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.nio.ByteBuffer password) throws org.apache.thrift.TException
     {
       changeLocalUserPassword_args args = new changeLocalUserPassword_args();
       args.setTinfo(tinfo);
@@ -495,13 +455,13 @@
     }
 
     @Override
-    public boolean authenticate(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws ThriftSecurityException, org.apache.thrift.TException
+    public boolean authenticate(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws ThriftSecurityException, org.apache.thrift.TException
     {
       send_authenticate(tinfo, credentials);
       return recv_authenticate();
     }
 
-    public void send_authenticate(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
+    public void send_authenticate(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
     {
       authenticate_args args = new authenticate_args();
       args.setTinfo(tinfo);
@@ -523,13 +483,13 @@
     }
 
     @Override
-    public boolean authenticateUser(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.securityImpl.thrift.TCredentials toAuth) throws ThriftSecurityException, org.apache.thrift.TException
+    public boolean authenticateUser(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.securityImpl.thrift.TCredentials toAuth) throws ThriftSecurityException, org.apache.thrift.TException
     {
       send_authenticateUser(tinfo, credentials, toAuth);
       return recv_authenticateUser();
     }
 
-    public void send_authenticateUser(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.securityImpl.thrift.TCredentials toAuth) throws org.apache.thrift.TException
+    public void send_authenticateUser(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.securityImpl.thrift.TCredentials toAuth) throws org.apache.thrift.TException
     {
       authenticateUser_args args = new authenticateUser_args();
       args.setTinfo(tinfo);
@@ -552,13 +512,13 @@
     }
 
     @Override
-    public void changeAuthorizations(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.util.List<java.nio.ByteBuffer> authorizations) throws ThriftSecurityException, org.apache.thrift.TException
+    public void changeAuthorizations(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.util.List<java.nio.ByteBuffer> authorizations) throws ThriftSecurityException, org.apache.thrift.TException
     {
       send_changeAuthorizations(tinfo, credentials, principal, authorizations);
       recv_changeAuthorizations();
     }
 
-    public void send_changeAuthorizations(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.util.List<java.nio.ByteBuffer> authorizations) throws org.apache.thrift.TException
+    public void send_changeAuthorizations(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.util.List<java.nio.ByteBuffer> authorizations) throws org.apache.thrift.TException
     {
       changeAuthorizations_args args = new changeAuthorizations_args();
       args.setTinfo(tinfo);
@@ -579,13 +539,13 @@
     }
 
     @Override
-    public java.util.List<java.nio.ByteBuffer> getUserAuthorizations(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal) throws ThriftSecurityException, org.apache.thrift.TException
+    public java.util.List<java.nio.ByteBuffer> getUserAuthorizations(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal) throws ThriftSecurityException, org.apache.thrift.TException
     {
       send_getUserAuthorizations(tinfo, credentials, principal);
       return recv_getUserAuthorizations();
     }
 
-    public void send_getUserAuthorizations(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal) throws org.apache.thrift.TException
+    public void send_getUserAuthorizations(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal) throws org.apache.thrift.TException
     {
       getUserAuthorizations_args args = new getUserAuthorizations_args();
       args.setTinfo(tinfo);
@@ -608,13 +568,13 @@
     }
 
     @Override
-    public boolean hasSystemPermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte sysPerm) throws ThriftSecurityException, org.apache.thrift.TException
+    public boolean hasSystemPermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte sysPerm) throws ThriftSecurityException, org.apache.thrift.TException
     {
       send_hasSystemPermission(tinfo, credentials, principal, sysPerm);
       return recv_hasSystemPermission();
     }
 
-    public void send_hasSystemPermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte sysPerm) throws org.apache.thrift.TException
+    public void send_hasSystemPermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte sysPerm) throws org.apache.thrift.TException
     {
       hasSystemPermission_args args = new hasSystemPermission_args();
       args.setTinfo(tinfo);
@@ -638,13 +598,13 @@
     }
 
     @Override
-    public boolean hasTablePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte tblPerm) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
+    public boolean hasTablePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte tblPerm) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
     {
       send_hasTablePermission(tinfo, credentials, principal, tableName, tblPerm);
       return recv_hasTablePermission();
     }
 
-    public void send_hasTablePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte tblPerm) throws org.apache.thrift.TException
+    public void send_hasTablePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte tblPerm) throws org.apache.thrift.TException
     {
       hasTablePermission_args args = new hasTablePermission_args();
       args.setTinfo(tinfo);
@@ -672,13 +632,13 @@
     }
 
     @Override
-    public boolean hasNamespacePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte tblNspcPerm) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
+    public boolean hasNamespacePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte tblNspcPerm) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
     {
       send_hasNamespacePermission(tinfo, credentials, principal, ns, tblNspcPerm);
       return recv_hasNamespacePermission();
     }
 
-    public void send_hasNamespacePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte tblNspcPerm) throws org.apache.thrift.TException
+    public void send_hasNamespacePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte tblNspcPerm) throws org.apache.thrift.TException
     {
       hasNamespacePermission_args args = new hasNamespacePermission_args();
       args.setTinfo(tinfo);
@@ -706,13 +666,13 @@
     }
 
     @Override
-    public void grantSystemPermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte permission) throws ThriftSecurityException, org.apache.thrift.TException
+    public void grantSystemPermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte permission) throws ThriftSecurityException, org.apache.thrift.TException
     {
       send_grantSystemPermission(tinfo, credentials, principal, permission);
       recv_grantSystemPermission();
     }
 
-    public void send_grantSystemPermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte permission) throws org.apache.thrift.TException
+    public void send_grantSystemPermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte permission) throws org.apache.thrift.TException
     {
       grantSystemPermission_args args = new grantSystemPermission_args();
       args.setTinfo(tinfo);
@@ -733,13 +693,13 @@
     }
 
     @Override
-    public void revokeSystemPermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte permission) throws ThriftSecurityException, org.apache.thrift.TException
+    public void revokeSystemPermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte permission) throws ThriftSecurityException, org.apache.thrift.TException
     {
       send_revokeSystemPermission(tinfo, credentials, principal, permission);
       recv_revokeSystemPermission();
     }
 
-    public void send_revokeSystemPermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte permission) throws org.apache.thrift.TException
+    public void send_revokeSystemPermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte permission) throws org.apache.thrift.TException
     {
       revokeSystemPermission_args args = new revokeSystemPermission_args();
       args.setTinfo(tinfo);
@@ -760,13 +720,13 @@
     }
 
     @Override
-    public void grantTablePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte permission) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
+    public void grantTablePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte permission) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
     {
       send_grantTablePermission(tinfo, credentials, principal, tableName, permission);
       recv_grantTablePermission();
     }
 
-    public void send_grantTablePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte permission) throws org.apache.thrift.TException
+    public void send_grantTablePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte permission) throws org.apache.thrift.TException
     {
       grantTablePermission_args args = new grantTablePermission_args();
       args.setTinfo(tinfo);
@@ -791,13 +751,13 @@
     }
 
     @Override
-    public void revokeTablePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte permission) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
+    public void revokeTablePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte permission) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
     {
       send_revokeTablePermission(tinfo, credentials, principal, tableName, permission);
       recv_revokeTablePermission();
     }
 
-    public void send_revokeTablePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte permission) throws org.apache.thrift.TException
+    public void send_revokeTablePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte permission) throws org.apache.thrift.TException
     {
       revokeTablePermission_args args = new revokeTablePermission_args();
       args.setTinfo(tinfo);
@@ -822,13 +782,13 @@
     }
 
     @Override
-    public void grantNamespacePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte permission) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
+    public void grantNamespacePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte permission) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
     {
       send_grantNamespacePermission(tinfo, credentials, principal, ns, permission);
       recv_grantNamespacePermission();
     }
 
-    public void send_grantNamespacePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte permission) throws org.apache.thrift.TException
+    public void send_grantNamespacePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte permission) throws org.apache.thrift.TException
     {
       grantNamespacePermission_args args = new grantNamespacePermission_args();
       args.setTinfo(tinfo);
@@ -853,13 +813,13 @@
     }
 
     @Override
-    public void revokeNamespacePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte permission) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
+    public void revokeNamespacePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte permission) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
     {
       send_revokeNamespacePermission(tinfo, credentials, principal, ns, permission);
       recv_revokeNamespacePermission();
     }
 
-    public void send_revokeNamespacePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte permission) throws org.apache.thrift.TException
+    public void send_revokeNamespacePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte permission) throws org.apache.thrift.TException
     {
       revokeNamespacePermission_args args = new revokeNamespacePermission_args();
       args.setTinfo(tinfo);
@@ -884,13 +844,13 @@
     }
 
     @Override
-    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 ThriftSecurityException, org.apache.thrift.TException
+    public java.util.Map<java.lang.String,java.lang.String> getConfiguration(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, ConfigurationType type) throws ThriftSecurityException, org.apache.thrift.TException
     {
       send_getConfiguration(tinfo, credentials, type);
       return recv_getConfiguration();
     }
 
-    public void send_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 void send_getConfiguration(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, ConfigurationType type) throws org.apache.thrift.TException
     {
       getConfiguration_args args = new getConfiguration_args();
       args.setTinfo(tinfo);
@@ -913,13 +873,13 @@
     }
 
     @Override
-    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 ThriftSecurityException, org.apache.thrift.TException
+    public java.util.Map<java.lang.String,java.lang.String> getSystemProperties(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws ThriftSecurityException, 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
+    public void send_getSystemProperties(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
     {
       getSystemProperties_args args = new getSystemProperties_args();
       args.setTinfo(tinfo);
@@ -941,13 +901,13 @@
     }
 
     @Override
-    public TVersionedProperties getVersionedSystemProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws ThriftSecurityException, org.apache.thrift.TException
+    public TVersionedProperties getVersionedSystemProperties(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws ThriftSecurityException, 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
+    public void send_getVersionedSystemProperties(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
     {
       getVersionedSystemProperties_args args = new getVersionedSystemProperties_args();
       args.setTinfo(tinfo);
@@ -969,13 +929,13 @@
     }
 
     @Override
-    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, ThriftSecurityException, org.apache.thrift.TException
+    public java.util.Map<java.lang.String,java.lang.String> getTableConfiguration(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName) throws ThriftTableOperationException, ThriftSecurityException, org.apache.thrift.TException
     {
       send_getTableConfiguration(tinfo, credentials, tableName);
       return recv_getTableConfiguration();
     }
 
-    public void send_getTableConfiguration(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
+    public void send_getTableConfiguration(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName) throws org.apache.thrift.TException
     {
       getTableConfiguration_args args = new getTableConfiguration_args();
       args.setTinfo(tinfo);
@@ -1001,13 +961,13 @@
     }
 
     @Override
-    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, ThriftSecurityException, org.apache.thrift.TException
+    public java.util.Map<java.lang.String,java.lang.String> getTableProperties(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName) throws ThriftTableOperationException, ThriftSecurityException, 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
+    public void send_getTableProperties(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);
@@ -1033,13 +993,13 @@
     }
 
     @Override
-    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, ThriftSecurityException, org.apache.thrift.TException
+    public TVersionedProperties getVersionedTableProperties(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName) throws ThriftTableOperationException, ThriftSecurityException, 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
+    public void send_getVersionedTableProperties(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);
@@ -1065,13 +1025,13 @@
     }
 
     @Override
-    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, ThriftSecurityException, org.apache.thrift.TException
+    public java.util.Map<java.lang.String,java.lang.String> getNamespaceConfiguration(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns) throws ThriftTableOperationException, ThriftSecurityException, org.apache.thrift.TException
     {
       send_getNamespaceConfiguration(tinfo, credentials, ns);
       return recv_getNamespaceConfiguration();
     }
 
-    public void send_getNamespaceConfiguration(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
+    public void send_getNamespaceConfiguration(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns) throws org.apache.thrift.TException
     {
       getNamespaceConfiguration_args args = new getNamespaceConfiguration_args();
       args.setTinfo(tinfo);
@@ -1097,13 +1057,13 @@
     }
 
     @Override
-    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, ThriftSecurityException, org.apache.thrift.TException
+    public java.util.Map<java.lang.String,java.lang.String> getNamespaceProperties(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns) throws ThriftTableOperationException, ThriftSecurityException, 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
+    public void send_getNamespaceProperties(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);
@@ -1129,13 +1089,13 @@
     }
 
     @Override
-    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, ThriftSecurityException, org.apache.thrift.TException
+    public TVersionedProperties getVersionedNamespaceProperties(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns) throws ThriftTableOperationException, ThriftSecurityException, 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
+    public void send_getVersionedNamespaceProperties(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);
@@ -1161,13 +1121,13 @@
     }
 
     @Override
-    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 checkClass(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);
       return recv_checkClass();
     }
 
-    public void send_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 void send_checkClass(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String className, java.lang.String interfaceMatch) throws org.apache.thrift.TException
     {
       checkClass_args args = new checkClass_args();
       args.setTinfo(tinfo);
@@ -1188,13 +1148,13 @@
     }
 
     @Override
-    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
+    public boolean checkTableClass(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
     {
       send_checkTableClass(tinfo, credentials, tableId, className, interfaceMatch);
       return recv_checkTableClass();
     }
 
-    public void send_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 org.apache.thrift.TException
+    public void send_checkTableClass(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableId, java.lang.String className, java.lang.String interfaceMatch) throws org.apache.thrift.TException
     {
       checkTableClass_args args = new checkTableClass_args();
       args.setTinfo(tinfo);
@@ -1222,13 +1182,13 @@
     }
 
     @Override
-    public boolean checkNamespaceClass(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String namespaceId, java.lang.String className, java.lang.String interfaceMatch) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
+    public boolean checkNamespaceClass(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String namespaceId, java.lang.String className, java.lang.String interfaceMatch) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
     {
       send_checkNamespaceClass(tinfo, credentials, namespaceId, className, interfaceMatch);
       return recv_checkNamespaceClass();
     }
 
-    public void send_checkNamespaceClass(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String namespaceId, java.lang.String className, java.lang.String interfaceMatch) throws org.apache.thrift.TException
+    public void send_checkNamespaceClass(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String namespaceId, java.lang.String className, java.lang.String interfaceMatch) throws org.apache.thrift.TException
     {
       checkNamespaceClass_args args = new checkNamespaceClass_args();
       args.setTinfo(tinfo);
@@ -1371,60 +1331,7 @@
     }
 
     @Override
-    public void bulkImportFiles(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long tid, java.lang.String tableId, java.util.List<java.lang.String> files, java.lang.String errorDir, boolean setTime, org.apache.thrift.async.AsyncMethodCallback<java.util.List<java.lang.String>> resultHandler) throws org.apache.thrift.TException {
-      checkReady();
-      bulkImportFiles_call method_call = new bulkImportFiles_call(tinfo, credentials, tid, tableId, files, errorDir, setTime, resultHandler, this, ___protocolFactory, ___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
-    }
-
-    public static class bulkImportFiles_call extends org.apache.thrift.async.TAsyncMethodCall<java.util.List<java.lang.String>> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
-      private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
-      private long tid;
-      private java.lang.String tableId;
-      private java.util.List<java.lang.String> files;
-      private java.lang.String errorDir;
-      private boolean setTime;
-      public bulkImportFiles_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long tid, java.lang.String tableId, java.util.List<java.lang.String> files, java.lang.String errorDir, boolean setTime, org.apache.thrift.async.AsyncMethodCallback<java.util.List<java.lang.String>> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
-        super(client, protocolFactory, transport, resultHandler, false);
-        this.tinfo = tinfo;
-        this.credentials = credentials;
-        this.tid = tid;
-        this.tableId = tableId;
-        this.files = files;
-        this.errorDir = errorDir;
-        this.setTime = setTime;
-      }
-
-      @Override
-      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("bulkImportFiles", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        bulkImportFiles_args args = new bulkImportFiles_args();
-        args.setTinfo(tinfo);
-        args.setCredentials(credentials);
-        args.setTid(tid);
-        args.setTableId(tableId);
-        args.setFiles(files);
-        args.setErrorDir(errorDir);
-        args.setSetTime(setTime);
-        args.write(prot);
-        prot.writeMessageEnd();
-      }
-
-      @Override
-      public java.util.List<java.lang.String> getResult() throws ThriftSecurityException, 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_bulkImportFiles();
-      }
-    }
-
-    @Override
-    public void isActive(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long tid, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
+    public void isActive(TInfo tinfo, long tid, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       isActive_call method_call = new isActive_call(tinfo, tid, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -1432,9 +1339,9 @@
     }
 
     public static class isActive_call extends org.apache.thrift.async.TAsyncMethodCall<java.lang.Boolean> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private TInfo tinfo;
       private long tid;
-      public isActive_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long tid, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public isActive_call(TInfo tinfo, long tid, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> 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.tid = tid;
@@ -1536,7 +1443,7 @@
     }
 
     @Override
-    public void listLocalUsers(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.Set<java.lang.String>> resultHandler) throws org.apache.thrift.TException {
+    public void listLocalUsers(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.Set<java.lang.String>> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       listLocalUsers_call method_call = new listLocalUsers_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -1544,9 +1451,9 @@
     }
 
     public static class listLocalUsers_call extends org.apache.thrift.async.TAsyncMethodCall<java.util.Set<java.lang.String>> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
-      public listLocalUsers_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.Set<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 {
+      public listLocalUsers_call(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.Set<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;
@@ -1574,7 +1481,7 @@
     }
 
     @Override
-    public void createLocalUser(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.nio.ByteBuffer password, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+    public void createLocalUser(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.nio.ByteBuffer password, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       createLocalUser_call method_call = new createLocalUser_call(tinfo, credentials, principal, password, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -1582,11 +1489,11 @@
     }
 
     public static class createLocalUser_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
       private java.lang.String principal;
       private java.nio.ByteBuffer password;
-      public createLocalUser_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.nio.ByteBuffer password, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public createLocalUser_call(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.nio.ByteBuffer password, org.apache.thrift.async.AsyncMethodCallback<Void> 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;
@@ -1619,7 +1526,7 @@
     }
 
     @Override
-    public void dropLocalUser(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+    public void dropLocalUser(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       dropLocalUser_call method_call = new dropLocalUser_call(tinfo, credentials, principal, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -1627,10 +1534,10 @@
     }
 
     public static class dropLocalUser_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
       private java.lang.String principal;
-      public dropLocalUser_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public dropLocalUser_call(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, org.apache.thrift.async.AsyncMethodCallback<Void> 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;
@@ -1661,7 +1568,7 @@
     }
 
     @Override
-    public void changeLocalUserPassword(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.nio.ByteBuffer password, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+    public void changeLocalUserPassword(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.nio.ByteBuffer password, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       changeLocalUserPassword_call method_call = new changeLocalUserPassword_call(tinfo, credentials, principal, password, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -1669,11 +1576,11 @@
     }
 
     public static class changeLocalUserPassword_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
       private java.lang.String principal;
       private java.nio.ByteBuffer password;
-      public changeLocalUserPassword_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.nio.ByteBuffer password, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public changeLocalUserPassword_call(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.nio.ByteBuffer password, org.apache.thrift.async.AsyncMethodCallback<Void> 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;
@@ -1706,7 +1613,7 @@
     }
 
     @Override
-    public void authenticate(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
+    public void authenticate(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       authenticate_call method_call = new authenticate_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -1714,9 +1621,9 @@
     }
 
     public static class authenticate_call extends org.apache.thrift.async.TAsyncMethodCall<java.lang.Boolean> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
-      public authenticate_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public authenticate_call(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> 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;
@@ -1744,7 +1651,7 @@
     }
 
     @Override
-    public void authenticateUser(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.securityImpl.thrift.TCredentials toAuth, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
+    public void authenticateUser(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.securityImpl.thrift.TCredentials toAuth, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       authenticateUser_call method_call = new authenticateUser_call(tinfo, credentials, toAuth, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -1752,10 +1659,10 @@
     }
 
     public static class authenticateUser_call extends org.apache.thrift.async.TAsyncMethodCall<java.lang.Boolean> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials toAuth;
-      public authenticateUser_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.securityImpl.thrift.TCredentials toAuth, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public authenticateUser_call(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.securityImpl.thrift.TCredentials toAuth, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> 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;
@@ -1785,7 +1692,7 @@
     }
 
     @Override
-    public void changeAuthorizations(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.util.List<java.nio.ByteBuffer> authorizations, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+    public void changeAuthorizations(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.util.List<java.nio.ByteBuffer> authorizations, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       changeAuthorizations_call method_call = new changeAuthorizations_call(tinfo, credentials, principal, authorizations, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -1793,11 +1700,11 @@
     }
 
     public static class changeAuthorizations_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
       private java.lang.String principal;
       private java.util.List<java.nio.ByteBuffer> authorizations;
-      public changeAuthorizations_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.util.List<java.nio.ByteBuffer> authorizations, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public changeAuthorizations_call(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.util.List<java.nio.ByteBuffer> authorizations, org.apache.thrift.async.AsyncMethodCallback<Void> 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;
@@ -1830,7 +1737,7 @@
     }
 
     @Override
-    public void getUserAuthorizations(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, org.apache.thrift.async.AsyncMethodCallback<java.util.List<java.nio.ByteBuffer>> resultHandler) throws org.apache.thrift.TException {
+    public void getUserAuthorizations(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, org.apache.thrift.async.AsyncMethodCallback<java.util.List<java.nio.ByteBuffer>> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       getUserAuthorizations_call method_call = new getUserAuthorizations_call(tinfo, credentials, principal, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -1838,10 +1745,10 @@
     }
 
     public static class getUserAuthorizations_call extends org.apache.thrift.async.TAsyncMethodCall<java.util.List<java.nio.ByteBuffer>> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
       private java.lang.String principal;
-      public getUserAuthorizations_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, org.apache.thrift.async.AsyncMethodCallback<java.util.List<java.nio.ByteBuffer>> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public getUserAuthorizations_call(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, org.apache.thrift.async.AsyncMethodCallback<java.util.List<java.nio.ByteBuffer>> 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;
@@ -1871,7 +1778,7 @@
     }
 
     @Override
-    public void hasSystemPermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte sysPerm, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
+    public void hasSystemPermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte sysPerm, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       hasSystemPermission_call method_call = new hasSystemPermission_call(tinfo, credentials, principal, sysPerm, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -1879,11 +1786,11 @@
     }
 
     public static class hasSystemPermission_call extends org.apache.thrift.async.TAsyncMethodCall<java.lang.Boolean> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
       private java.lang.String principal;
       private byte sysPerm;
-      public hasSystemPermission_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte sysPerm, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public hasSystemPermission_call(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte sysPerm, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> 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;
@@ -1915,7 +1822,7 @@
     }
 
     @Override
-    public void hasTablePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte tblPerm, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
+    public void hasTablePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte tblPerm, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       hasTablePermission_call method_call = new hasTablePermission_call(tinfo, credentials, principal, tableName, tblPerm, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -1923,12 +1830,12 @@
     }
 
     public static class hasTablePermission_call extends org.apache.thrift.async.TAsyncMethodCall<java.lang.Boolean> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
       private java.lang.String principal;
       private java.lang.String tableName;
       private byte tblPerm;
-      public hasTablePermission_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte tblPerm, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public hasTablePermission_call(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte tblPerm, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> 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;
@@ -1962,7 +1869,7 @@
     }
 
     @Override
-    public void hasNamespacePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte tblNspcPerm, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
+    public void hasNamespacePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte tblNspcPerm, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       hasNamespacePermission_call method_call = new hasNamespacePermission_call(tinfo, credentials, principal, ns, tblNspcPerm, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -1970,12 +1877,12 @@
     }
 
     public static class hasNamespacePermission_call extends org.apache.thrift.async.TAsyncMethodCall<java.lang.Boolean> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
       private java.lang.String principal;
       private java.lang.String ns;
       private byte tblNspcPerm;
-      public hasNamespacePermission_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte tblNspcPerm, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public hasNamespacePermission_call(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte tblNspcPerm, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> 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;
@@ -2009,7 +1916,7 @@
     }
 
     @Override
-    public void grantSystemPermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+    public void grantSystemPermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       grantSystemPermission_call method_call = new grantSystemPermission_call(tinfo, credentials, principal, permission, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -2017,11 +1924,11 @@
     }
 
     public static class grantSystemPermission_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
       private java.lang.String principal;
       private byte permission;
-      public grantSystemPermission_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public grantSystemPermission_call(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> 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;
@@ -2054,7 +1961,7 @@
     }
 
     @Override
-    public void revokeSystemPermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+    public void revokeSystemPermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       revokeSystemPermission_call method_call = new revokeSystemPermission_call(tinfo, credentials, principal, permission, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -2062,11 +1969,11 @@
     }
 
     public static class revokeSystemPermission_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
       private java.lang.String principal;
       private byte permission;
-      public revokeSystemPermission_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public revokeSystemPermission_call(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> 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;
@@ -2099,7 +2006,7 @@
     }
 
     @Override
-    public void grantTablePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+    public void grantTablePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       grantTablePermission_call method_call = new grantTablePermission_call(tinfo, credentials, principal, tableName, permission, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -2107,12 +2014,12 @@
     }
 
     public static class grantTablePermission_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
       private java.lang.String principal;
       private java.lang.String tableName;
       private byte permission;
-      public grantTablePermission_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public grantTablePermission_call(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> 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;
@@ -2147,7 +2054,7 @@
     }
 
     @Override
-    public void revokeTablePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+    public void revokeTablePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       revokeTablePermission_call method_call = new revokeTablePermission_call(tinfo, credentials, principal, tableName, permission, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -2155,12 +2062,12 @@
     }
 
     public static class revokeTablePermission_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
       private java.lang.String principal;
       private java.lang.String tableName;
       private byte permission;
-      public revokeTablePermission_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public revokeTablePermission_call(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> 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;
@@ -2195,7 +2102,7 @@
     }
 
     @Override
-    public void grantNamespacePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+    public void grantNamespacePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       grantNamespacePermission_call method_call = new grantNamespacePermission_call(tinfo, credentials, principal, ns, permission, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -2203,12 +2110,12 @@
     }
 
     public static class grantNamespacePermission_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
       private java.lang.String principal;
       private java.lang.String ns;
       private byte permission;
-      public grantNamespacePermission_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public grantNamespacePermission_call(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> 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;
@@ -2243,7 +2150,7 @@
     }
 
     @Override
-    public void revokeNamespacePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+    public void revokeNamespacePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       revokeNamespacePermission_call method_call = new revokeNamespacePermission_call(tinfo, credentials, principal, ns, permission, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -2251,12 +2158,12 @@
     }
 
     public static class revokeNamespacePermission_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
       private java.lang.String principal;
       private java.lang.String ns;
       private byte permission;
-      public revokeNamespacePermission_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public revokeNamespacePermission_call(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> 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;
@@ -2291,7 +2198,7 @@
     }
 
     @Override
-    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 getConfiguration(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 {
       checkReady();
       getConfiguration_call method_call = new getConfiguration_call(tinfo, credentials, type, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -2299,10 +2206,10 @@
     }
 
     public static class getConfiguration_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 TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
       private ConfigurationType type;
-      public getConfiguration_call(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, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public getConfiguration_call(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, 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;
@@ -2332,7 +2239,7 @@
     }
 
     @Override
-    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 getSystemProperties(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;
@@ -2340,9 +2247,9 @@
     }
 
     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 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 {
+      public getSystemProperties_call(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;
@@ -2370,7 +2277,7 @@
     }
 
     @Override
-    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 getVersionedSystemProperties(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;
@@ -2378,9 +2285,9 @@
     }
 
     public static class getVersionedSystemProperties_call extends org.apache.thrift.async.TAsyncMethodCall<TVersionedProperties> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private 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 {
+      public getVersionedSystemProperties_call(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;
@@ -2408,7 +2315,7 @@
     }
 
     @Override
-    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 getTableConfiguration(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);
       this.___currentMethod = method_call;
@@ -2416,10 +2323,10 @@
     }
 
     public static class getTableConfiguration_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 TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
       private java.lang.String tableName;
-      public getTableConfiguration_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.TException {
+      public getTableConfiguration_call(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.TException {
         super(client, protocolFactory, transport, resultHandler, false);
         this.tinfo = tinfo;
         this.credentials = credentials;
@@ -2449,7 +2356,7 @@
     }
 
     @Override
-    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 getTableProperties(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;
@@ -2457,10 +2364,10 @@
     }
 
     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 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.TException {
+      public getTableProperties_call(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.TException {
         super(client, protocolFactory, transport, resultHandler, false);
         this.tinfo = tinfo;
         this.credentials = credentials;
@@ -2490,7 +2397,7 @@
     }
 
     @Override
-    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 getVersionedTableProperties(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;
@@ -2498,10 +2405,10 @@
     }
 
     public static class getVersionedTableProperties_call extends org.apache.thrift.async.TAsyncMethodCall<TVersionedProperties> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private 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 {
+      public getVersionedTableProperties_call(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;
@@ -2531,7 +2438,7 @@
     }
 
     @Override
-    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 getNamespaceConfiguration(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);
       this.___currentMethod = method_call;
@@ -2539,10 +2446,10 @@
     }
 
     public static class getNamespaceConfiguration_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 TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
       private java.lang.String ns;
-      public getNamespaceConfiguration_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 {
+      public getNamespaceConfiguration_call(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;
@@ -2572,7 +2479,7 @@
     }
 
     @Override
-    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 getNamespaceProperties(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;
@@ -2580,10 +2487,10 @@
     }
 
     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 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 {
+      public getNamespaceProperties_call(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;
@@ -2613,7 +2520,7 @@
     }
 
     @Override
-    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 getVersionedNamespaceProperties(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;
@@ -2621,10 +2528,10 @@
     }
 
     public static class getVersionedNamespaceProperties_call extends org.apache.thrift.async.TAsyncMethodCall<TVersionedProperties> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private 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 {
+      public getVersionedNamespaceProperties_call(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;
@@ -2654,7 +2561,7 @@
     }
 
     @Override
-    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 checkClass(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);
       this.___currentMethod = method_call;
@@ -2662,11 +2569,11 @@
     }
 
     public static class checkClass_call extends org.apache.thrift.async.TAsyncMethodCall<java.lang.Boolean> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
       private java.lang.String className;
       private java.lang.String interfaceMatch;
-      public checkClass_call(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, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public checkClass_call(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, 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;
@@ -2698,7 +2605,7 @@
     }
 
     @Override
-    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 {
+    public void checkTableClass(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 {
       checkReady();
       checkTableClass_call method_call = new checkTableClass_call(tinfo, credentials, tableId, className, interfaceMatch, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -2706,12 +2613,12 @@
     }
 
     public static class checkTableClass_call extends org.apache.thrift.async.TAsyncMethodCall<java.lang.Boolean> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
       private java.lang.String tableId;
       private java.lang.String className;
       private java.lang.String interfaceMatch;
-      public checkTableClass_call(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, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public checkTableClass_call(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, 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;
@@ -2745,7 +2652,7 @@
     }
 
     @Override
-    public void checkNamespaceClass(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String namespaceId, java.lang.String className, java.lang.String interfaceMatch, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
+    public void checkNamespaceClass(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String namespaceId, java.lang.String className, java.lang.String interfaceMatch, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       checkNamespaceClass_call method_call = new checkNamespaceClass_call(tinfo, credentials, namespaceId, className, interfaceMatch, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -2753,12 +2660,12 @@
     }
 
     public static class checkNamespaceClass_call extends org.apache.thrift.async.TAsyncMethodCall<java.lang.Boolean> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
       private java.lang.String namespaceId;
       private java.lang.String className;
       private java.lang.String interfaceMatch;
-      public checkNamespaceClass_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String namespaceId, java.lang.String className, java.lang.String interfaceMatch, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public checkNamespaceClass_call(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String namespaceId, java.lang.String className, java.lang.String interfaceMatch, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> 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;
@@ -2807,7 +2714,6 @@
       processMap.put("getRootTabletLocation", new getRootTabletLocation());
       processMap.put("getInstanceId", new getInstanceId());
       processMap.put("getZooKeepers", new getZooKeepers());
-      processMap.put("bulkImportFiles", new bulkImportFiles());
       processMap.put("isActive", new isActive());
       processMap.put("ping", new ping());
       processMap.put("getDiskUsage", new getDiskUsage());
@@ -2927,40 +2833,6 @@
       }
     }
 
-    public static class bulkImportFiles<I extends Iface> extends org.apache.thrift.ProcessFunction<I, bulkImportFiles_args> {
-      public bulkImportFiles() {
-        super("bulkImportFiles");
-      }
-
-      @Override
-      public bulkImportFiles_args getEmptyArgsInstance() {
-        return new bulkImportFiles_args();
-      }
-
-      @Override
-      protected boolean isOneway() {
-        return false;
-      }
-
-      @Override
-      protected boolean rethrowUnhandledExceptions() {
-        return false;
-      }
-
-      @Override
-      public bulkImportFiles_result getResult(I iface, bulkImportFiles_args args) throws org.apache.thrift.TException {
-        bulkImportFiles_result result = new bulkImportFiles_result();
-        try {
-          result.success = iface.bulkImportFiles(args.tinfo, args.credentials, args.tid, args.tableId, args.files, args.errorDir, args.setTime);
-        } catch (ThriftSecurityException sec) {
-          result.sec = sec;
-        } catch (ThriftTableOperationException tope) {
-          result.tope = tope;
-        }
-        return result;
-      }
-    }
-
     public static class isActive<I extends Iface> extends org.apache.thrift.ProcessFunction<I, isActive_args> {
       public isActive() {
         super("isActive");
@@ -4032,7 +3904,6 @@
       processMap.put("getRootTabletLocation", new getRootTabletLocation());
       processMap.put("getInstanceId", new getInstanceId());
       processMap.put("getZooKeepers", new getZooKeepers());
-      processMap.put("bulkImportFiles", new bulkImportFiles());
       processMap.put("isActive", new isActive());
       processMap.put("ping", new ping());
       processMap.put("getDiskUsage", new getDiskUsage());
@@ -4269,81 +4140,6 @@
       }
     }
 
-    public static class bulkImportFiles<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, bulkImportFiles_args, java.util.List<java.lang.String>> {
-      public bulkImportFiles() {
-        super("bulkImportFiles");
-      }
-
-      @Override
-      public bulkImportFiles_args getEmptyArgsInstance() {
-        return new bulkImportFiles_args();
-      }
-
-      @Override
-      public org.apache.thrift.async.AsyncMethodCallback<java.util.List<java.lang.String>> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
-        final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new org.apache.thrift.async.AsyncMethodCallback<java.util.List<java.lang.String>>() { 
-          @Override
-          public void onComplete(java.util.List<java.lang.String> o) {
-            bulkImportFiles_result result = new bulkImportFiles_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);
-            }
-          }
-          @Override
-          public void onError(java.lang.Exception e) {
-            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
-            org.apache.thrift.TSerializable msg;
-            bulkImportFiles_result result = new bulkImportFiles_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();
-            }
-          }
-        };
-      }
-
-      @Override
-      protected boolean isOneway() {
-        return false;
-      }
-
-      @Override
-      public void start(I iface, bulkImportFiles_args args, org.apache.thrift.async.AsyncMethodCallback<java.util.List<java.lang.String>> resultHandler) throws org.apache.thrift.TException {
-        iface.bulkImportFiles(args.tinfo, args.credentials, args.tid, args.tableId, args.files, args.errorDir, args.setTime,resultHandler);
-      }
-    }
-
     public static class isActive<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, isActive_args, java.lang.Boolean> {
       public isActive() {
         super("isActive");
@@ -8601,1703 +8397,6 @@
   }
 
   @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-  public static class bulkImportFiles_args implements org.apache.thrift.TBase<bulkImportFiles_args, bulkImportFiles_args._Fields>, java.io.Serializable, Cloneable, Comparable<bulkImportFiles_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("bulkImportFiles_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)8);
-    private static final org.apache.thrift.protocol.TField TID_FIELD_DESC = new org.apache.thrift.protocol.TField("tid", org.apache.thrift.protocol.TType.I64, (short)3);
-    private static final org.apache.thrift.protocol.TField TABLE_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("tableId", org.apache.thrift.protocol.TType.STRING, (short)4);
-    private static final org.apache.thrift.protocol.TField FILES_FIELD_DESC = new org.apache.thrift.protocol.TField("files", org.apache.thrift.protocol.TType.LIST, (short)5);
-    private static final org.apache.thrift.protocol.TField ERROR_DIR_FIELD_DESC = new org.apache.thrift.protocol.TField("errorDir", org.apache.thrift.protocol.TType.STRING, (short)6);
-    private static final org.apache.thrift.protocol.TField SET_TIME_FIELD_DESC = new org.apache.thrift.protocol.TField("setTime", org.apache.thrift.protocol.TType.BOOL, (short)7);
-
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new bulkImportFiles_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new bulkImportFiles_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 long tid; // required
-    public @org.apache.thrift.annotation.Nullable java.lang.String tableId; // required
-    public @org.apache.thrift.annotation.Nullable java.util.List<java.lang.String> files; // required
-    public @org.apache.thrift.annotation.Nullable java.lang.String errorDir; // required
-    public boolean setTime; // 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)8, "credentials"),
-      TID((short)3, "tid"),
-      TABLE_ID((short)4, "tableId"),
-      FILES((short)5, "files"),
-      ERROR_DIR((short)6, "errorDir"),
-      SET_TIME((short)7, "setTime");
-
-      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 8: // CREDENTIALS
-            return CREDENTIALS;
-          case 3: // TID
-            return TID;
-          case 4: // TABLE_ID
-            return TABLE_ID;
-          case 5: // FILES
-            return FILES;
-          case 6: // ERROR_DIR
-            return ERROR_DIR;
-          case 7: // SET_TIME
-            return SET_TIME;
-          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;
-      }
-
-      @Override
-      public short getThriftFieldId() {
-        return _thriftId;
-      }
-
-      @Override
-      public java.lang.String getFieldName() {
-        return _fieldName;
-      }
-    }
-
-    // isset id assignments
-    private static final int __TID_ISSET_ID = 0;
-    private static final int __SETTIME_ISSET_ID = 1;
-    private byte __isset_bitfield = 0;
-    public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
-    static {
-      java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.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.TID, new org.apache.thrift.meta_data.FieldMetaData("tid", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
-      tmpMap.put(_Fields.TABLE_ID, new org.apache.thrift.meta_data.FieldMetaData("tableId", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
-      tmpMap.put(_Fields.FILES, new org.apache.thrift.meta_data.FieldMetaData("files", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
-              new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
-      tmpMap.put(_Fields.ERROR_DIR, new org.apache.thrift.meta_data.FieldMetaData("errorDir", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
-      tmpMap.put(_Fields.SET_TIME, new org.apache.thrift.meta_data.FieldMetaData("setTime", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
-      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(bulkImportFiles_args.class, metaDataMap);
-    }
-
-    public bulkImportFiles_args() {
-    }
-
-    public bulkImportFiles_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
-      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
-      long tid,
-      java.lang.String tableId,
-      java.util.List<java.lang.String> files,
-      java.lang.String errorDir,
-      boolean setTime)
-    {
-      this();
-      this.tinfo = tinfo;
-      this.credentials = credentials;
-      this.tid = tid;
-      setTidIsSet(true);
-      this.tableId = tableId;
-      this.files = files;
-      this.errorDir = errorDir;
-      this.setTime = setTime;
-      setSetTimeIsSet(true);
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public bulkImportFiles_args(bulkImportFiles_args other) {
-      __isset_bitfield = other.__isset_bitfield;
-      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);
-      }
-      this.tid = other.tid;
-      if (other.isSetTableId()) {
-        this.tableId = other.tableId;
-      }
-      if (other.isSetFiles()) {
-        java.util.List<java.lang.String> __this__files = new java.util.ArrayList<java.lang.String>(other.files);
-        this.files = __this__files;
-      }
-      if (other.isSetErrorDir()) {
-        this.errorDir = other.errorDir;
-      }
-      this.setTime = other.setTime;
-    }
-
-    @Override
-    public bulkImportFiles_args deepCopy() {
-      return new bulkImportFiles_args(this);
-    }
-
-    @Override
-    public void clear() {
-      this.tinfo = null;
-      this.credentials = null;
-      setTidIsSet(false);
-      this.tid = 0;
-      this.tableId = null;
-      this.files = null;
-      this.errorDir = null;
-      setSetTimeIsSet(false);
-      this.setTime = false;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
-      return this.tinfo;
-    }
-
-    public bulkImportFiles_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 bulkImportFiles_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 long getTid() {
-      return this.tid;
-    }
-
-    public bulkImportFiles_args setTid(long tid) {
-      this.tid = tid;
-      setTidIsSet(true);
-      return this;
-    }
-
-    public void unsetTid() {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __TID_ISSET_ID);
-    }
-
-    /** Returns true if field tid is set (has been assigned a value) and false otherwise */
-    public boolean isSetTid() {
-      return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __TID_ISSET_ID);
-    }
-
-    public void setTidIsSet(boolean value) {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __TID_ISSET_ID, value);
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.lang.String getTableId() {
-      return this.tableId;
-    }
-
-    public bulkImportFiles_args setTableId(@org.apache.thrift.annotation.Nullable java.lang.String tableId) {
-      this.tableId = tableId;
-      return this;
-    }
-
-    public void unsetTableId() {
-      this.tableId = null;
-    }
-
-    /** Returns true if field tableId is set (has been assigned a value) and false otherwise */
-    public boolean isSetTableId() {
-      return this.tableId != null;
-    }
-
-    public void setTableIdIsSet(boolean value) {
-      if (!value) {
-        this.tableId = null;
-      }
-    }
-
-    public int getFilesSize() {
-      return (this.files == null) ? 0 : this.files.size();
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.util.Iterator<java.lang.String> getFilesIterator() {
-      return (this.files == null) ? null : this.files.iterator();
-    }
-
-    public void addToFiles(java.lang.String elem) {
-      if (this.files == null) {
-        this.files = new java.util.ArrayList<java.lang.String>();
-      }
-      this.files.add(elem);
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.util.List<java.lang.String> getFiles() {
-      return this.files;
-    }
-
-    public bulkImportFiles_args setFiles(@org.apache.thrift.annotation.Nullable java.util.List<java.lang.String> files) {
-      this.files = files;
-      return this;
-    }
-
-    public void unsetFiles() {
-      this.files = null;
-    }
-
-    /** Returns true if field files is set (has been assigned a value) and false otherwise */
-    public boolean isSetFiles() {
-      return this.files != null;
-    }
-
-    public void setFilesIsSet(boolean value) {
-      if (!value) {
-        this.files = null;
-      }
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.lang.String getErrorDir() {
-      return this.errorDir;
-    }
-
-    public bulkImportFiles_args setErrorDir(@org.apache.thrift.annotation.Nullable java.lang.String errorDir) {
-      this.errorDir = errorDir;
-      return this;
-    }
-
-    public void unsetErrorDir() {
-      this.errorDir = null;
-    }
-
-    /** Returns true if field errorDir is set (has been assigned a value) and false otherwise */
-    public boolean isSetErrorDir() {
-      return this.errorDir != null;
-    }
-
-    public void setErrorDirIsSet(boolean value) {
-      if (!value) {
-        this.errorDir = null;
-      }
-    }
-
-    public boolean isSetTime() {
-      return this.setTime;
-    }
-
-    public bulkImportFiles_args setSetTime(boolean setTime) {
-      this.setTime = setTime;
-      setSetTimeIsSet(true);
-      return this;
-    }
-
-    public void unsetSetTime() {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __SETTIME_ISSET_ID);
-    }
-
-    /** Returns true if field setTime is set (has been assigned a value) and false otherwise */
-    public boolean isSetSetTime() {
-      return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __SETTIME_ISSET_ID);
-    }
-
-    public void setSetTimeIsSet(boolean value) {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __SETTIME_ISSET_ID, value);
-    }
-
-    @Override
-    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 TID:
-        if (value == null) {
-          unsetTid();
-        } else {
-          setTid((java.lang.Long)value);
-        }
-        break;
-
-      case TABLE_ID:
-        if (value == null) {
-          unsetTableId();
-        } else {
-          setTableId((java.lang.String)value);
-        }
-        break;
-
-      case FILES:
-        if (value == null) {
-          unsetFiles();
-        } else {
-          setFiles((java.util.List<java.lang.String>)value);
-        }
-        break;
-
-      case ERROR_DIR:
-        if (value == null) {
-          unsetErrorDir();
-        } else {
-          setErrorDir((java.lang.String)value);
-        }
-        break;
-
-      case SET_TIME:
-        if (value == null) {
-          unsetSetTime();
-        } else {
-          setSetTime((java.lang.Boolean)value);
-        }
-        break;
-
-      }
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    @Override
-    public java.lang.Object getFieldValue(_Fields field) {
-      switch (field) {
-      case TINFO:
-        return getTinfo();
-
-      case CREDENTIALS:
-        return getCredentials();
-
-      case TID:
-        return getTid();
-
-      case TABLE_ID:
-        return getTableId();
-
-      case FILES:
-        return getFiles();
-
-      case ERROR_DIR:
-        return getErrorDir();
-
-      case SET_TIME:
-        return isSetTime();
-
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
-    @Override
-    public boolean isSet(_Fields field) {
-      if (field == null) {
-        throw new java.lang.IllegalArgumentException();
-      }
-
-      switch (field) {
-      case TINFO:
-        return isSetTinfo();
-      case CREDENTIALS:
-        return isSetCredentials();
-      case TID:
-        return isSetTid();
-      case TABLE_ID:
-        return isSetTableId();
-      case FILES:
-        return isSetFiles();
-      case ERROR_DIR:
-        return isSetErrorDir();
-      case SET_TIME:
-        return isSetSetTime();
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(java.lang.Object that) {
-      if (that instanceof bulkImportFiles_args)
-        return this.equals((bulkImportFiles_args)that);
-      return false;
-    }
-
-    public boolean equals(bulkImportFiles_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_tid = true;
-      boolean that_present_tid = true;
-      if (this_present_tid || that_present_tid) {
-        if (!(this_present_tid && that_present_tid))
-          return false;
-        if (this.tid != that.tid)
-          return false;
-      }
-
-      boolean this_present_tableId = true && this.isSetTableId();
-      boolean that_present_tableId = true && that.isSetTableId();
-      if (this_present_tableId || that_present_tableId) {
-        if (!(this_present_tableId && that_present_tableId))
-          return false;
-        if (!this.tableId.equals(that.tableId))
-          return false;
-      }
-
-      boolean this_present_files = true && this.isSetFiles();
-      boolean that_present_files = true && that.isSetFiles();
-      if (this_present_files || that_present_files) {
-        if (!(this_present_files && that_present_files))
-          return false;
-        if (!this.files.equals(that.files))
-          return false;
-      }
-
-      boolean this_present_errorDir = true && this.isSetErrorDir();
-      boolean that_present_errorDir = true && that.isSetErrorDir();
-      if (this_present_errorDir || that_present_errorDir) {
-        if (!(this_present_errorDir && that_present_errorDir))
-          return false;
-        if (!this.errorDir.equals(that.errorDir))
-          return false;
-      }
-
-      boolean this_present_setTime = true;
-      boolean that_present_setTime = true;
-      if (this_present_setTime || that_present_setTime) {
-        if (!(this_present_setTime && that_present_setTime))
-          return false;
-        if (this.setTime != that.setTime)
-          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 + org.apache.thrift.TBaseHelper.hashCode(tid);
-
-      hashCode = hashCode * 8191 + ((isSetTableId()) ? 131071 : 524287);
-      if (isSetTableId())
-        hashCode = hashCode * 8191 + tableId.hashCode();
-
-      hashCode = hashCode * 8191 + ((isSetFiles()) ? 131071 : 524287);
-      if (isSetFiles())
-        hashCode = hashCode * 8191 + files.hashCode();
-
-      hashCode = hashCode * 8191 + ((isSetErrorDir()) ? 131071 : 524287);
-      if (isSetErrorDir())
-        hashCode = hashCode * 8191 + errorDir.hashCode();
-
-      hashCode = hashCode * 8191 + ((setTime) ? 131071 : 524287);
-
-      return hashCode;
-    }
-
-    @Override
-    public int compareTo(bulkImportFiles_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(isSetTid(), other.isSetTid());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetTid()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tid, other.tid);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = java.lang.Boolean.compare(isSetTableId(), other.isSetTableId());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetTableId()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableId, other.tableId);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = java.lang.Boolean.compare(isSetFiles(), other.isSetFiles());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetFiles()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.files, other.files);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = java.lang.Boolean.compare(isSetErrorDir(), other.isSetErrorDir());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetErrorDir()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.errorDir, other.errorDir);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = java.lang.Boolean.compare(isSetSetTime(), other.isSetSetTime());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetSetTime()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.setTime, other.setTime);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      return 0;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    @Override
-    public _Fields fieldForId(int fieldId) {
-      return _Fields.findByThriftId(fieldId);
-    }
-
-    @Override
-    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
-      scheme(iprot).read(iprot, this);
-    }
-
-    @Override
-    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("bulkImportFiles_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("tid:");
-      sb.append(this.tid);
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("tableId:");
-      if (this.tableId == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.tableId);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("files:");
-      if (this.files == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.files);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("errorDir:");
-      if (this.errorDir == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.errorDir);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("setTime:");
-      sb.append(this.setTime);
-      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 {
-        // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
-        __isset_bitfield = 0;
-        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
-      } catch (org.apache.thrift.TException te) {
-        throw new java.io.IOException(te);
-      }
-    }
-
-    private static class bulkImportFiles_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public bulkImportFiles_argsStandardScheme getScheme() {
-        return new bulkImportFiles_argsStandardScheme();
-      }
-    }
-
-    private static class bulkImportFiles_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<bulkImportFiles_args> {
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol iprot, bulkImportFiles_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 8: // 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 3: // TID
-              if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
-                struct.tid = iprot.readI64();
-                struct.setTidIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 4: // TABLE_ID
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
-                struct.tableId = iprot.readString();
-                struct.setTableIdIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 5: // FILES
-              if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
-                {
-                  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)
-                  {
-                    _elem19 = iprot.readString();
-                    struct.files.add(_elem19);
-                  }
-                  iprot.readListEnd();
-                }
-                struct.setFilesIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 6: // ERROR_DIR
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
-                struct.errorDir = iprot.readString();
-                struct.setErrorDirIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 7: // SET_TIME
-              if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
-                struct.setTime = iprot.readBool();
-                struct.setSetTimeIsSet(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();
-      }
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol oprot, bulkImportFiles_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();
-        }
-        oprot.writeFieldBegin(TID_FIELD_DESC);
-        oprot.writeI64(struct.tid);
-        oprot.writeFieldEnd();
-        if (struct.tableId != null) {
-          oprot.writeFieldBegin(TABLE_ID_FIELD_DESC);
-          oprot.writeString(struct.tableId);
-          oprot.writeFieldEnd();
-        }
-        if (struct.files != null) {
-          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 _iter21 : struct.files)
-            {
-              oprot.writeString(_iter21);
-            }
-            oprot.writeListEnd();
-          }
-          oprot.writeFieldEnd();
-        }
-        if (struct.errorDir != null) {
-          oprot.writeFieldBegin(ERROR_DIR_FIELD_DESC);
-          oprot.writeString(struct.errorDir);
-          oprot.writeFieldEnd();
-        }
-        oprot.writeFieldBegin(SET_TIME_FIELD_DESC);
-        oprot.writeBool(struct.setTime);
-        oprot.writeFieldEnd();
-        if (struct.credentials != null) {
-          oprot.writeFieldBegin(CREDENTIALS_FIELD_DESC);
-          struct.credentials.write(oprot);
-          oprot.writeFieldEnd();
-        }
-        oprot.writeFieldStop();
-        oprot.writeStructEnd();
-      }
-
-    }
-
-    private static class bulkImportFiles_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public bulkImportFiles_argsTupleScheme getScheme() {
-        return new bulkImportFiles_argsTupleScheme();
-      }
-    }
-
-    private static class bulkImportFiles_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<bulkImportFiles_args> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, bulkImportFiles_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.isSetTid()) {
-          optionals.set(2);
-        }
-        if (struct.isSetTableId()) {
-          optionals.set(3);
-        }
-        if (struct.isSetFiles()) {
-          optionals.set(4);
-        }
-        if (struct.isSetErrorDir()) {
-          optionals.set(5);
-        }
-        if (struct.isSetSetTime()) {
-          optionals.set(6);
-        }
-        oprot.writeBitSet(optionals, 7);
-        if (struct.isSetTinfo()) {
-          struct.tinfo.write(oprot);
-        }
-        if (struct.isSetCredentials()) {
-          struct.credentials.write(oprot);
-        }
-        if (struct.isSetTid()) {
-          oprot.writeI64(struct.tid);
-        }
-        if (struct.isSetTableId()) {
-          oprot.writeString(struct.tableId);
-        }
-        if (struct.isSetFiles()) {
-          {
-            oprot.writeI32(struct.files.size());
-            for (java.lang.String _iter22 : struct.files)
-            {
-              oprot.writeString(_iter22);
-            }
-          }
-        }
-        if (struct.isSetErrorDir()) {
-          oprot.writeString(struct.errorDir);
-        }
-        if (struct.isSetSetTime()) {
-          oprot.writeBool(struct.setTime);
-        }
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, bulkImportFiles_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(7);
-        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.tid = iprot.readI64();
-          struct.setTidIsSet(true);
-        }
-        if (incoming.get(3)) {
-          struct.tableId = iprot.readString();
-          struct.setTableIdIsSet(true);
-        }
-        if (incoming.get(4)) {
-          {
-            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)
-            {
-              _elem24 = iprot.readString();
-              struct.files.add(_elem24);
-            }
-          }
-          struct.setFilesIsSet(true);
-        }
-        if (incoming.get(5)) {
-          struct.errorDir = iprot.readString();
-          struct.setErrorDirIsSet(true);
-        }
-        if (incoming.get(6)) {
-          struct.setTime = iprot.readBool();
-          struct.setSetTimeIsSet(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();
-    }
-  }
-
-  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-  public static class bulkImportFiles_result implements org.apache.thrift.TBase<bulkImportFiles_result, bulkImportFiles_result._Fields>, java.io.Serializable, Cloneable, Comparable<bulkImportFiles_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("bulkImportFiles_result");
-
-    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.LIST, (short)0);
-    private static final org.apache.thrift.protocol.TField SEC_FIELD_DESC = new org.apache.thrift.protocol.TField("sec", org.apache.thrift.protocol.TType.STRUCT, (short)1);
-    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)2);
-
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new bulkImportFiles_resultStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new bulkImportFiles_resultTupleSchemeFactory();
-
-    public @org.apache.thrift.annotation.Nullable java.util.List<java.lang.String> success; // required
-    public @org.apache.thrift.annotation.Nullable ThriftSecurityException sec; // 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"),
-      SEC((short)1, "sec"),
-      TOPE((short)2, "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: // SEC
-            return SEC;
-          case 2: // 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;
-      }
-
-      @Override
-      public short getThriftFieldId() {
-        return _thriftId;
-      }
-
-      @Override
-      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.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
-              new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
-      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(bulkImportFiles_result.class, metaDataMap);
-    }
-
-    public bulkImportFiles_result() {
-    }
-
-    public bulkImportFiles_result(
-      java.util.List<java.lang.String> success,
-      ThriftSecurityException sec,
-      ThriftTableOperationException tope)
-    {
-      this();
-      this.success = success;
-      this.sec = sec;
-      this.tope = tope;
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public bulkImportFiles_result(bulkImportFiles_result other) {
-      if (other.isSetSuccess()) {
-        java.util.List<java.lang.String> __this__success = new java.util.ArrayList<java.lang.String>(other.success);
-        this.success = __this__success;
-      }
-      if (other.isSetSec()) {
-        this.sec = new ThriftSecurityException(other.sec);
-      }
-      if (other.isSetTope()) {
-        this.tope = new ThriftTableOperationException(other.tope);
-      }
-    }
-
-    @Override
-    public bulkImportFiles_result deepCopy() {
-      return new bulkImportFiles_result(this);
-    }
-
-    @Override
-    public void clear() {
-      this.success = null;
-      this.sec = null;
-      this.tope = null;
-    }
-
-    public int getSuccessSize() {
-      return (this.success == null) ? 0 : this.success.size();
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.util.Iterator<java.lang.String> getSuccessIterator() {
-      return (this.success == null) ? null : this.success.iterator();
-    }
-
-    public void addToSuccess(java.lang.String elem) {
-      if (this.success == null) {
-        this.success = new java.util.ArrayList<java.lang.String>();
-      }
-      this.success.add(elem);
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.util.List<java.lang.String> getSuccess() {
-      return this.success;
-    }
-
-    public bulkImportFiles_result setSuccess(@org.apache.thrift.annotation.Nullable java.util.List<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 ThriftSecurityException getSec() {
-      return this.sec;
-    }
-
-    public bulkImportFiles_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 bulkImportFiles_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;
-      }
-    }
-
-    @Override
-    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.List<java.lang.String>)value);
-        }
-        break;
-
-      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
-    @Override
-    public java.lang.Object getFieldValue(_Fields field) {
-      switch (field) {
-      case SUCCESS:
-        return getSuccess();
-
-      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 */
-    @Override
-    public boolean isSet(_Fields field) {
-      if (field == null) {
-        throw new java.lang.IllegalArgumentException();
-      }
-
-      switch (field) {
-      case SUCCESS:
-        return isSetSuccess();
-      case SEC:
-        return isSetSec();
-      case TOPE:
-        return isSetTope();
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(java.lang.Object that) {
-      if (that instanceof bulkImportFiles_result)
-        return this.equals((bulkImportFiles_result)that);
-      return false;
-    }
-
-    public boolean equals(bulkImportFiles_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_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 + ((isSetSuccess()) ? 131071 : 524287);
-      if (isSetSuccess())
-        hashCode = hashCode * 8191 + success.hashCode();
-
-      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(bulkImportFiles_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(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
-    @Override
-    public _Fields fieldForId(int fieldId) {
-      return _Fields.findByThriftId(fieldId);
-    }
-
-    @Override
-    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("bulkImportFiles_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("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 bulkImportFiles_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public bulkImportFiles_resultStandardScheme getScheme() {
-        return new bulkImportFiles_resultStandardScheme();
-      }
-    }
-
-    private static class bulkImportFiles_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<bulkImportFiles_result> {
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol iprot, bulkImportFiles_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.LIST) {
-                {
-                  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)
-                  {
-                    _elem27 = iprot.readString();
-                    struct.success.add(_elem27);
-                  }
-                  iprot.readListEnd();
-                }
-                struct.setSuccessIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            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();
-      }
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol oprot, bulkImportFiles_result struct) throws org.apache.thrift.TException {
-        struct.validate();
-
-        oprot.writeStructBegin(STRUCT_DESC);
-        if (struct.success != null) {
-          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 _iter29 : struct.success)
-            {
-              oprot.writeString(_iter29);
-            }
-            oprot.writeListEnd();
-          }
-          oprot.writeFieldEnd();
-        }
-        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 bulkImportFiles_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public bulkImportFiles_resultTupleScheme getScheme() {
-        return new bulkImportFiles_resultTupleScheme();
-      }
-    }
-
-    private static class bulkImportFiles_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<bulkImportFiles_result> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, bulkImportFiles_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.isSetSec()) {
-          optionals.set(1);
-        }
-        if (struct.isSetTope()) {
-          optionals.set(2);
-        }
-        oprot.writeBitSet(optionals, 3);
-        if (struct.isSetSuccess()) {
-          {
-            oprot.writeI32(struct.success.size());
-            for (java.lang.String _iter30 : struct.success)
-            {
-              oprot.writeString(_iter30);
-            }
-          }
-        }
-        if (struct.isSetSec()) {
-          struct.sec.write(oprot);
-        }
-        if (struct.isSetTope()) {
-          struct.tope.write(oprot);
-        }
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, bulkImportFiles_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(3);
-        if (incoming.get(0)) {
-          {
-            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)
-            {
-              _elem32 = iprot.readString();
-              struct.success.add(_elem32);
-            }
-          }
-          struct.setSuccessIsSet(true);
-        }
-        if (incoming.get(1)) {
-          struct.sec = new ThriftSecurityException();
-          struct.sec.read(iprot);
-          struct.setSecIsSet(true);
-        }
-        if (incoming.get(2)) {
-          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();
-    }
-  }
-
-  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
   public static class isActive_args implements org.apache.thrift.TBase<isActive_args, isActive_args._Fields>, java.io.Serializable, Cloneable, Comparable<isActive_args>   {
     private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("isActive_args");
 
@@ -10307,7 +8406,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new isActive_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new isActive_argsTupleSchemeFactory();
 
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+    public @org.apache.thrift.annotation.Nullable TInfo tinfo; // required
     public long tid; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
@@ -10382,7 +8481,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TInfo.class)));
       tmpMap.put(_Fields.TID, new org.apache.thrift.meta_data.FieldMetaData("tid", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
@@ -10393,7 +8492,7 @@
     }
 
     public isActive_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      TInfo tinfo,
       long tid)
     {
       this();
@@ -10408,7 +8507,7 @@
     public isActive_args(isActive_args other) {
       __isset_bitfield = other.__isset_bitfield;
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new TInfo(other.tinfo);
       }
       this.tid = other.tid;
     }
@@ -10426,11 +8525,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public isActive_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public isActive_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -10480,7 +8579,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((TInfo)value);
         }
         break;
 
@@ -10687,7 +8786,7 @@
           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 = new TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -10765,7 +8864,7 @@
         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 = new TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -12332,13 +10431,13 @@
             case 2: // TABLES
               if (schemeField.type == org.apache.thrift.protocol.TType.SET) {
                 {
-                  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)
+                  org.apache.thrift.protocol.TSet _set28 = iprot.readSetBegin();
+                  struct.tables = new java.util.HashSet<java.lang.String>(2*_set28.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _elem29;
+                  for (int _i30 = 0; _i30 < _set28.size; ++_i30)
                   {
-                    _elem35 = iprot.readString();
-                    struct.tables.add(_elem35);
+                    _elem29 = iprot.readString();
+                    struct.tables.add(_elem29);
                   }
                   iprot.readSetEnd();
                 }
@@ -12381,9 +10480,9 @@
           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 _iter37 : struct.tables)
+            for (java.lang.String _iter31 : struct.tables)
             {
-              oprot.writeString(_iter37);
+              oprot.writeString(_iter31);
             }
             oprot.writeSetEnd();
           }
@@ -12418,9 +10517,9 @@
         if (struct.isSetTables()) {
           {
             oprot.writeI32(struct.tables.size());
-            for (java.lang.String _iter38 : struct.tables)
+            for (java.lang.String _iter32 : struct.tables)
             {
-              oprot.writeString(_iter38);
+              oprot.writeString(_iter32);
             }
           }
         }
@@ -12435,13 +10534,13 @@
         java.util.BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            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)
+            org.apache.thrift.protocol.TSet _set33 = iprot.readSetBegin(org.apache.thrift.protocol.TType.STRING);
+            struct.tables = new java.util.HashSet<java.lang.String>(2*_set33.size);
+            @org.apache.thrift.annotation.Nullable java.lang.String _elem34;
+            for (int _i35 = 0; _i35 < _set33.size; ++_i35)
             {
-              _elem40 = iprot.readString();
-              struct.tables.add(_elem40);
+              _elem34 = iprot.readString();
+              struct.tables.add(_elem34);
             }
           }
           struct.setTablesIsSet(true);
@@ -12952,14 +11051,14 @@
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  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)
+                  org.apache.thrift.protocol.TList _list36 = iprot.readListBegin();
+                  struct.success = new java.util.ArrayList<TDiskUsage>(_list36.size);
+                  @org.apache.thrift.annotation.Nullable TDiskUsage _elem37;
+                  for (int _i38 = 0; _i38 < _list36.size; ++_i38)
                   {
-                    _elem43 = new TDiskUsage();
-                    _elem43.read(iprot);
-                    struct.success.add(_elem43);
+                    _elem37 = new TDiskUsage();
+                    _elem37.read(iprot);
+                    struct.success.add(_elem37);
                   }
                   iprot.readListEnd();
                 }
@@ -13006,9 +11105,9 @@
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (TDiskUsage _iter45 : struct.success)
+            for (TDiskUsage _iter39 : struct.success)
             {
-              _iter45.write(oprot);
+              _iter39.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -13056,9 +11155,9 @@
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (TDiskUsage _iter46 : struct.success)
+            for (TDiskUsage _iter40 : struct.success)
             {
-              _iter46.write(oprot);
+              _iter40.write(oprot);
             }
           }
         }
@@ -13076,14 +11175,14 @@
         java.util.BitSet incoming = iprot.readBitSet(3);
         if (incoming.get(0)) {
           {
-            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)
+            org.apache.thrift.protocol.TList _list41 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+            struct.success = new java.util.ArrayList<TDiskUsage>(_list41.size);
+            @org.apache.thrift.annotation.Nullable TDiskUsage _elem42;
+            for (int _i43 = 0; _i43 < _list41.size; ++_i43)
             {
-              _elem48 = new TDiskUsage();
-              _elem48.read(iprot);
-              struct.success.add(_elem48);
+              _elem42 = new TDiskUsage();
+              _elem42.read(iprot);
+              struct.success.add(_elem42);
             }
           }
           struct.setSuccessIsSet(true);
@@ -13116,7 +11215,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new listLocalUsers_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new listLocalUsers_argsTupleSchemeFactory();
 
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+    public @org.apache.thrift.annotation.Nullable 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. */
@@ -13189,7 +11288,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, 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);
@@ -13200,7 +11299,7 @@
     }
 
     public listLocalUsers_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials)
     {
       this();
@@ -13213,7 +11312,7 @@
      */
     public listLocalUsers_args(listLocalUsers_args other) {
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -13232,11 +11331,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public listLocalUsers_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public listLocalUsers_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -13288,7 +11387,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((TInfo)value);
         }
         break;
 
@@ -13502,7 +11601,7 @@
           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 = new TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -13583,7 +11682,7 @@
         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 = new TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -14008,13 +12107,13 @@
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.SET) {
                 {
-                  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)
+                  org.apache.thrift.protocol.TSet _set44 = iprot.readSetBegin();
+                  struct.success = new java.util.HashSet<java.lang.String>(2*_set44.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _elem45;
+                  for (int _i46 = 0; _i46 < _set44.size; ++_i46)
                   {
-                    _elem51 = iprot.readString();
-                    struct.success.add(_elem51);
+                    _elem45 = iprot.readString();
+                    struct.success.add(_elem45);
                   }
                   iprot.readSetEnd();
                 }
@@ -14052,9 +12151,9 @@
           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 _iter53 : struct.success)
+            for (java.lang.String _iter47 : struct.success)
             {
-              oprot.writeString(_iter53);
+              oprot.writeString(_iter47);
             }
             oprot.writeSetEnd();
           }
@@ -14094,9 +12193,9 @@
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (java.lang.String _iter54 : struct.success)
+            for (java.lang.String _iter48 : struct.success)
             {
-              oprot.writeString(_iter54);
+              oprot.writeString(_iter48);
             }
           }
         }
@@ -14111,13 +12210,13 @@
         java.util.BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            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)
+            org.apache.thrift.protocol.TSet _set49 = iprot.readSetBegin(org.apache.thrift.protocol.TType.STRING);
+            struct.success = new java.util.HashSet<java.lang.String>(2*_set49.size);
+            @org.apache.thrift.annotation.Nullable java.lang.String _elem50;
+            for (int _i51 = 0; _i51 < _set49.size; ++_i51)
             {
-              _elem56 = iprot.readString();
-              struct.success.add(_elem56);
+              _elem50 = iprot.readString();
+              struct.success.add(_elem50);
             }
           }
           struct.setSuccessIsSet(true);
@@ -14147,7 +12246,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new createLocalUser_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new createLocalUser_argsTupleSchemeFactory();
 
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+    public @org.apache.thrift.annotation.Nullable 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 principal; // required
     public @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer password; // required
@@ -14228,7 +12327,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, 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.PRINCIPAL, new org.apache.thrift.meta_data.FieldMetaData("principal", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -14243,7 +12342,7 @@
     }
 
     public createLocalUser_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
       java.lang.String principal,
       java.nio.ByteBuffer password)
@@ -14260,7 +12359,7 @@
      */
     public createLocalUser_args(createLocalUser_args other) {
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -14287,11 +12386,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public createLocalUser_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public createLocalUser_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -14402,7 +12501,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((TInfo)value);
         }
         break;
 
@@ -14708,7 +12807,7 @@
           switch (schemeField.id) {
             case 5: // TINFO
               if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+                struct.tinfo = new TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -14827,7 +12926,7 @@
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
-          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+          struct.tinfo = new TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -15243,7 +13342,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new dropLocalUser_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new dropLocalUser_argsTupleSchemeFactory();
 
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+    public @org.apache.thrift.annotation.Nullable 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 principal; // required
 
@@ -15320,7 +13419,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, 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.PRINCIPAL, new org.apache.thrift.meta_data.FieldMetaData("principal", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -15333,7 +13432,7 @@
     }
 
     public dropLocalUser_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
       java.lang.String principal)
     {
@@ -15348,7 +13447,7 @@
      */
     public dropLocalUser_args(dropLocalUser_args other) {
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -15371,11 +13470,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public dropLocalUser_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public dropLocalUser_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -15452,7 +13551,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((TInfo)value);
         }
         break;
 
@@ -15710,7 +13809,7 @@
           switch (schemeField.id) {
             case 3: // TINFO
               if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+                struct.tinfo = new TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -15810,7 +13909,7 @@
         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 = new TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -16223,7 +14322,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new changeLocalUserPassword_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new changeLocalUserPassword_argsTupleSchemeFactory();
 
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+    public @org.apache.thrift.annotation.Nullable 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 principal; // required
     public @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer password; // required
@@ -16304,7 +14403,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, 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.PRINCIPAL, new org.apache.thrift.meta_data.FieldMetaData("principal", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -16319,7 +14418,7 @@
     }
 
     public changeLocalUserPassword_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
       java.lang.String principal,
       java.nio.ByteBuffer password)
@@ -16336,7 +14435,7 @@
      */
     public changeLocalUserPassword_args(changeLocalUserPassword_args other) {
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -16363,11 +14462,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public changeLocalUserPassword_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public changeLocalUserPassword_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -16478,7 +14577,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((TInfo)value);
         }
         break;
 
@@ -16784,7 +14883,7 @@
           switch (schemeField.id) {
             case 4: // TINFO
               if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+                struct.tinfo = new TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -16903,7 +15002,7 @@
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
-          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+          struct.tinfo = new TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -17318,7 +15417,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new authenticate_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new authenticate_argsTupleSchemeFactory();
 
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+    public @org.apache.thrift.annotation.Nullable 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. */
@@ -17391,7 +15490,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, 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);
@@ -17402,7 +15501,7 @@
     }
 
     public authenticate_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials)
     {
       this();
@@ -17415,7 +15514,7 @@
      */
     public authenticate_args(authenticate_args other) {
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -17434,11 +15533,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public authenticate_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public authenticate_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -17490,7 +15589,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((TInfo)value);
         }
         break;
 
@@ -17704,7 +15803,7 @@
           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 = new TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -17785,7 +15884,7 @@
         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 = new TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -18295,7 +16394,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new authenticateUser_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new authenticateUser_argsTupleSchemeFactory();
 
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+    public @org.apache.thrift.annotation.Nullable TInfo tinfo; // required
     public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
     public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials toAuth; // required
 
@@ -18372,7 +16471,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, 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.TO_AUTH, new org.apache.thrift.meta_data.FieldMetaData("toAuth", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -18385,7 +16484,7 @@
     }
 
     public authenticateUser_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials toAuth)
     {
@@ -18400,7 +16499,7 @@
      */
     public authenticateUser_args(authenticateUser_args other) {
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -18423,11 +16522,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public authenticateUser_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public authenticateUser_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -18504,7 +16603,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((TInfo)value);
         }
         break;
 
@@ -18765,7 +16864,7 @@
           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 = new TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -18866,7 +16965,7 @@
         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 = new TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -19382,7 +17481,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new changeAuthorizations_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new changeAuthorizations_argsTupleSchemeFactory();
 
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+    public @org.apache.thrift.annotation.Nullable 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 principal; // required
     public @org.apache.thrift.annotation.Nullable java.util.List<java.nio.ByteBuffer> authorizations; // required
@@ -19463,7 +17562,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, 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.PRINCIPAL, new org.apache.thrift.meta_data.FieldMetaData("principal", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -19479,7 +17578,7 @@
     }
 
     public changeAuthorizations_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
       java.lang.String principal,
       java.util.List<java.nio.ByteBuffer> authorizations)
@@ -19496,7 +17595,7 @@
      */
     public changeAuthorizations_args(changeAuthorizations_args other) {
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -19524,11 +17623,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public changeAuthorizations_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public changeAuthorizations_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -19646,7 +17745,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((TInfo)value);
         }
         break;
 
@@ -19948,7 +18047,7 @@
           switch (schemeField.id) {
             case 4: // TINFO
               if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+                struct.tinfo = new TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -19975,13 +18074,13 @@
             case 3: // AUTHORIZATIONS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  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)
+                  org.apache.thrift.protocol.TList _list52 = iprot.readListBegin();
+                  struct.authorizations = new java.util.ArrayList<java.nio.ByteBuffer>(_list52.size);
+                  @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem53;
+                  for (int _i54 = 0; _i54 < _list52.size; ++_i54)
                   {
-                    _elem59 = iprot.readBinary();
-                    struct.authorizations.add(_elem59);
+                    _elem53 = iprot.readBinary();
+                    struct.authorizations.add(_elem53);
                   }
                   iprot.readListEnd();
                 }
@@ -20015,9 +18114,9 @@
           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 _iter61 : struct.authorizations)
+            for (java.nio.ByteBuffer _iter55 : struct.authorizations)
             {
-              oprot.writeBinary(_iter61);
+              oprot.writeBinary(_iter55);
             }
             oprot.writeListEnd();
           }
@@ -20077,9 +18176,9 @@
         if (struct.isSetAuthorizations()) {
           {
             oprot.writeI32(struct.authorizations.size());
-            for (java.nio.ByteBuffer _iter62 : struct.authorizations)
+            for (java.nio.ByteBuffer _iter56 : struct.authorizations)
             {
-              oprot.writeBinary(_iter62);
+              oprot.writeBinary(_iter56);
             }
           }
         }
@@ -20090,7 +18189,7 @@
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
-          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+          struct.tinfo = new TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -20105,13 +18204,13 @@
         }
         if (incoming.get(3)) {
           {
-            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)
+            org.apache.thrift.protocol.TList _list57 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
+            struct.authorizations = new java.util.ArrayList<java.nio.ByteBuffer>(_list57.size);
+            @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem58;
+            for (int _i59 = 0; _i59 < _list57.size; ++_i59)
             {
-              _elem64 = iprot.readBinary();
-              struct.authorizations.add(_elem64);
+              _elem58 = iprot.readBinary();
+              struct.authorizations.add(_elem58);
             }
           }
           struct.setAuthorizationsIsSet(true);
@@ -20515,7 +18614,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getUserAuthorizations_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getUserAuthorizations_argsTupleSchemeFactory();
 
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+    public @org.apache.thrift.annotation.Nullable 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 principal; // required
 
@@ -20592,7 +18691,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, 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.PRINCIPAL, new org.apache.thrift.meta_data.FieldMetaData("principal", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -20605,7 +18704,7 @@
     }
 
     public getUserAuthorizations_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
       java.lang.String principal)
     {
@@ -20620,7 +18719,7 @@
      */
     public getUserAuthorizations_args(getUserAuthorizations_args other) {
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -20643,11 +18742,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public getUserAuthorizations_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public getUserAuthorizations_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -20724,7 +18823,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((TInfo)value);
         }
         break;
 
@@ -20982,7 +19081,7 @@
           switch (schemeField.id) {
             case 3: // TINFO
               if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+                struct.tinfo = new TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -21082,7 +19181,7 @@
         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 = new TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -21511,13 +19610,13 @@
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  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)
+                  org.apache.thrift.protocol.TList _list60 = iprot.readListBegin();
+                  struct.success = new java.util.ArrayList<java.nio.ByteBuffer>(_list60.size);
+                  @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem61;
+                  for (int _i62 = 0; _i62 < _list60.size; ++_i62)
                   {
-                    _elem67 = iprot.readBinary();
-                    struct.success.add(_elem67);
+                    _elem61 = iprot.readBinary();
+                    struct.success.add(_elem61);
                   }
                   iprot.readListEnd();
                 }
@@ -21555,9 +19654,9 @@
           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 _iter69 : struct.success)
+            for (java.nio.ByteBuffer _iter63 : struct.success)
             {
-              oprot.writeBinary(_iter69);
+              oprot.writeBinary(_iter63);
             }
             oprot.writeListEnd();
           }
@@ -21597,9 +19696,9 @@
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (java.nio.ByteBuffer _iter70 : struct.success)
+            for (java.nio.ByteBuffer _iter64 : struct.success)
             {
-              oprot.writeBinary(_iter70);
+              oprot.writeBinary(_iter64);
             }
           }
         }
@@ -21614,13 +19713,13 @@
         java.util.BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            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)
+            org.apache.thrift.protocol.TList _list65 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
+            struct.success = new java.util.ArrayList<java.nio.ByteBuffer>(_list65.size);
+            @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem66;
+            for (int _i67 = 0; _i67 < _list65.size; ++_i67)
             {
-              _elem72 = iprot.readBinary();
-              struct.success.add(_elem72);
+              _elem66 = iprot.readBinary();
+              struct.success.add(_elem66);
             }
           }
           struct.setSuccessIsSet(true);
@@ -21650,7 +19749,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new hasSystemPermission_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new hasSystemPermission_argsTupleSchemeFactory();
 
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+    public @org.apache.thrift.annotation.Nullable 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 principal; // required
     public byte sysPerm; // required
@@ -21733,7 +19832,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, 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.PRINCIPAL, new org.apache.thrift.meta_data.FieldMetaData("principal", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -21748,7 +19847,7 @@
     }
 
     public hasSystemPermission_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
       java.lang.String principal,
       byte sysPerm)
@@ -21767,7 +19866,7 @@
     public hasSystemPermission_args(hasSystemPermission_args other) {
       __isset_bitfield = other.__isset_bitfield;
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -21793,11 +19892,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public hasSystemPermission_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public hasSystemPermission_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -21897,7 +19996,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((TInfo)value);
         }
         break;
 
@@ -22195,7 +20294,7 @@
           switch (schemeField.id) {
             case 4: // TINFO
               if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+                struct.tinfo = new TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -22312,7 +20411,7 @@
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
-          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+          struct.tinfo = new TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -22832,7 +20931,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new hasTablePermission_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new hasTablePermission_argsTupleSchemeFactory();
 
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+    public @org.apache.thrift.annotation.Nullable 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 principal; // required
     public @org.apache.thrift.annotation.Nullable java.lang.String tableName; // required
@@ -22919,7 +21018,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, 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.PRINCIPAL, new org.apache.thrift.meta_data.FieldMetaData("principal", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -22936,7 +21035,7 @@
     }
 
     public hasTablePermission_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
       java.lang.String principal,
       java.lang.String tableName,
@@ -22957,7 +21056,7 @@
     public hasTablePermission_args(hasTablePermission_args other) {
       __isset_bitfield = other.__isset_bitfield;
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -22987,11 +21086,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public hasTablePermission_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public hasTablePermission_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -23116,7 +21215,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((TInfo)value);
         }
         break;
 
@@ -23458,7 +21557,7 @@
           switch (schemeField.id) {
             case 5: // TINFO
               if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+                struct.tinfo = new TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -23594,7 +21693,7 @@
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet incoming = iprot.readBitSet(5);
         if (incoming.get(0)) {
-          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+          struct.tinfo = new TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -24225,7 +22324,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new hasNamespacePermission_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new hasNamespacePermission_argsTupleSchemeFactory();
 
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+    public @org.apache.thrift.annotation.Nullable 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 principal; // required
     public @org.apache.thrift.annotation.Nullable java.lang.String ns; // required
@@ -24312,7 +22411,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, 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.PRINCIPAL, new org.apache.thrift.meta_data.FieldMetaData("principal", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -24329,7 +22428,7 @@
     }
 
     public hasNamespacePermission_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
       java.lang.String principal,
       java.lang.String ns,
@@ -24350,7 +22449,7 @@
     public hasNamespacePermission_args(hasNamespacePermission_args other) {
       __isset_bitfield = other.__isset_bitfield;
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -24380,11 +22479,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public hasNamespacePermission_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public hasNamespacePermission_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -24509,7 +22608,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((TInfo)value);
         }
         break;
 
@@ -24851,7 +22950,7 @@
           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 = new TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -24987,7 +23086,7 @@
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet incoming = iprot.readBitSet(5);
         if (incoming.get(0)) {
-          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+          struct.tinfo = new TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -25617,7 +23716,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new grantSystemPermission_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new grantSystemPermission_argsTupleSchemeFactory();
 
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+    public @org.apache.thrift.annotation.Nullable 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 principal; // required
     public byte permission; // required
@@ -25700,7 +23799,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, 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.PRINCIPAL, new org.apache.thrift.meta_data.FieldMetaData("principal", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -25715,7 +23814,7 @@
     }
 
     public grantSystemPermission_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
       java.lang.String principal,
       byte permission)
@@ -25734,7 +23833,7 @@
     public grantSystemPermission_args(grantSystemPermission_args other) {
       __isset_bitfield = other.__isset_bitfield;
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -25760,11 +23859,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public grantSystemPermission_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public grantSystemPermission_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -25864,7 +23963,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((TInfo)value);
         }
         break;
 
@@ -26162,7 +24261,7 @@
           switch (schemeField.id) {
             case 4: // TINFO
               if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+                struct.tinfo = new TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -26279,7 +24378,7 @@
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
-          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+          struct.tinfo = new TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -26696,7 +24795,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new revokeSystemPermission_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new revokeSystemPermission_argsTupleSchemeFactory();
 
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+    public @org.apache.thrift.annotation.Nullable 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 principal; // required
     public byte permission; // required
@@ -26779,7 +24878,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, 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.PRINCIPAL, new org.apache.thrift.meta_data.FieldMetaData("principal", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -26794,7 +24893,7 @@
     }
 
     public revokeSystemPermission_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
       java.lang.String principal,
       byte permission)
@@ -26813,7 +24912,7 @@
     public revokeSystemPermission_args(revokeSystemPermission_args other) {
       __isset_bitfield = other.__isset_bitfield;
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -26839,11 +24938,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public revokeSystemPermission_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public revokeSystemPermission_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -26943,7 +25042,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((TInfo)value);
         }
         break;
 
@@ -27241,7 +25340,7 @@
           switch (schemeField.id) {
             case 4: // TINFO
               if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+                struct.tinfo = new TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -27358,7 +25457,7 @@
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
-          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+          struct.tinfo = new TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -27776,7 +25875,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new grantTablePermission_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new grantTablePermission_argsTupleSchemeFactory();
 
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+    public @org.apache.thrift.annotation.Nullable 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 principal; // required
     public @org.apache.thrift.annotation.Nullable java.lang.String tableName; // required
@@ -27863,7 +25962,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, 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.PRINCIPAL, new org.apache.thrift.meta_data.FieldMetaData("principal", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -27880,7 +25979,7 @@
     }
 
     public grantTablePermission_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
       java.lang.String principal,
       java.lang.String tableName,
@@ -27901,7 +26000,7 @@
     public grantTablePermission_args(grantTablePermission_args other) {
       __isset_bitfield = other.__isset_bitfield;
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -27931,11 +26030,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public grantTablePermission_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public grantTablePermission_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -28060,7 +26159,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((TInfo)value);
         }
         break;
 
@@ -28402,7 +26501,7 @@
           switch (schemeField.id) {
             case 5: // TINFO
               if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+                struct.tinfo = new TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -28538,7 +26637,7 @@
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet incoming = iprot.readBitSet(5);
         if (incoming.get(0)) {
-          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+          struct.tinfo = new TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -29067,7 +27166,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new revokeTablePermission_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new revokeTablePermission_argsTupleSchemeFactory();
 
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+    public @org.apache.thrift.annotation.Nullable 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 principal; // required
     public @org.apache.thrift.annotation.Nullable java.lang.String tableName; // required
@@ -29154,7 +27253,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, 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.PRINCIPAL, new org.apache.thrift.meta_data.FieldMetaData("principal", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -29171,7 +27270,7 @@
     }
 
     public revokeTablePermission_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
       java.lang.String principal,
       java.lang.String tableName,
@@ -29192,7 +27291,7 @@
     public revokeTablePermission_args(revokeTablePermission_args other) {
       __isset_bitfield = other.__isset_bitfield;
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -29222,11 +27321,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public revokeTablePermission_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public revokeTablePermission_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -29351,7 +27450,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((TInfo)value);
         }
         break;
 
@@ -29693,7 +27792,7 @@
           switch (schemeField.id) {
             case 5: // TINFO
               if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+                struct.tinfo = new TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -29829,7 +27928,7 @@
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet incoming = iprot.readBitSet(5);
         if (incoming.get(0)) {
-          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+          struct.tinfo = new TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -30358,7 +28457,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new grantNamespacePermission_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new grantNamespacePermission_argsTupleSchemeFactory();
 
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+    public @org.apache.thrift.annotation.Nullable 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 principal; // required
     public @org.apache.thrift.annotation.Nullable java.lang.String ns; // required
@@ -30445,7 +28544,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, 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.PRINCIPAL, new org.apache.thrift.meta_data.FieldMetaData("principal", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -30462,7 +28561,7 @@
     }
 
     public grantNamespacePermission_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
       java.lang.String principal,
       java.lang.String ns,
@@ -30483,7 +28582,7 @@
     public grantNamespacePermission_args(grantNamespacePermission_args other) {
       __isset_bitfield = other.__isset_bitfield;
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -30513,11 +28612,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public grantNamespacePermission_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public grantNamespacePermission_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -30642,7 +28741,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((TInfo)value);
         }
         break;
 
@@ -30984,7 +29083,7 @@
           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 = new TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -31120,7 +29219,7 @@
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet incoming = iprot.readBitSet(5);
         if (incoming.get(0)) {
-          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+          struct.tinfo = new TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -31649,7 +29748,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new revokeNamespacePermission_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new revokeNamespacePermission_argsTupleSchemeFactory();
 
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+    public @org.apache.thrift.annotation.Nullable 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 principal; // required
     public @org.apache.thrift.annotation.Nullable java.lang.String ns; // required
@@ -31736,7 +29835,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, 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.PRINCIPAL, new org.apache.thrift.meta_data.FieldMetaData("principal", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -31753,7 +29852,7 @@
     }
 
     public revokeNamespacePermission_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
       java.lang.String principal,
       java.lang.String ns,
@@ -31774,7 +29873,7 @@
     public revokeNamespacePermission_args(revokeNamespacePermission_args other) {
       __isset_bitfield = other.__isset_bitfield;
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -31804,11 +29903,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public revokeNamespacePermission_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public revokeNamespacePermission_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -31933,7 +30032,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((TInfo)value);
         }
         break;
 
@@ -32275,7 +30374,7 @@
           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 = new TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -32411,7 +30510,7 @@
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet incoming = iprot.readBitSet(5);
         if (incoming.get(0)) {
-          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+          struct.tinfo = new TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -32938,7 +31037,7 @@
     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 TInfo tinfo; // required
     public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
     /**
      * 
@@ -33023,7 +31122,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, 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, 
@@ -33036,7 +31135,7 @@
     }
 
     public getConfiguration_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
       ConfigurationType type)
     {
@@ -33051,7 +31150,7 @@
      */
     public getConfiguration_args(getConfiguration_args other) {
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -33074,11 +31173,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public getConfiguration_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public getConfiguration_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -33163,7 +31262,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((TInfo)value);
         }
         break;
 
@@ -33421,7 +31520,7 @@
           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 = new TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -33521,7 +31620,7 @@
         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 = new TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -33946,15 +32045,15 @@
             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)
+                  org.apache.thrift.protocol.TMap _map68 = iprot.readMapBegin();
+                  struct.success = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map68.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _key69;
+                  @org.apache.thrift.annotation.Nullable java.lang.String _val70;
+                  for (int _i71 = 0; _i71 < _map68.size; ++_i71)
                   {
-                    _key75 = iprot.readString();
-                    _val76 = iprot.readString();
-                    struct.success.put(_key75, _val76);
+                    _key69 = iprot.readString();
+                    _val70 = iprot.readString();
+                    struct.success.put(_key69, _val70);
                   }
                   iprot.readMapEnd();
                 }
@@ -33992,10 +32091,10 @@
           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())
+            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter72 : struct.success.entrySet())
             {
-              oprot.writeString(_iter78.getKey());
-              oprot.writeString(_iter78.getValue());
+              oprot.writeString(_iter72.getKey());
+              oprot.writeString(_iter72.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -34035,10 +32134,10 @@
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter79 : struct.success.entrySet())
+            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter73 : struct.success.entrySet())
             {
-              oprot.writeString(_iter79.getKey());
-              oprot.writeString(_iter79.getValue());
+              oprot.writeString(_iter73.getKey());
+              oprot.writeString(_iter73.getValue());
             }
           }
         }
@@ -34053,15 +32152,15 @@
         java.util.BitSet incoming = iprot.readBitSet(2);
         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)
+            org.apache.thrift.protocol.TMap _map74 = 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*_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)
             {
-              _key81 = iprot.readString();
-              _val82 = iprot.readString();
-              struct.success.put(_key81, _val82);
+              _key75 = iprot.readString();
+              _val76 = iprot.readString();
+              struct.success.put(_key75, _val76);
             }
           }
           struct.setSuccessIsSet(true);
@@ -34089,7 +32188,7 @@
     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 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. */
@@ -34162,7 +32261,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, 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);
@@ -34173,7 +32272,7 @@
     }
 
     public getSystemProperties_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials)
     {
       this();
@@ -34186,7 +32285,7 @@
      */
     public getSystemProperties_args(getSystemProperties_args other) {
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -34205,11 +32304,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public getSystemProperties_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public getSystemProperties_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -34261,7 +32360,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((TInfo)value);
         }
         break;
 
@@ -34475,7 +32574,7 @@
           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 = new TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -34556,7 +32655,7 @@
         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 = new TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -34977,15 +33076,15 @@
             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)
+                  org.apache.thrift.protocol.TMap _map78 = iprot.readMapBegin();
+                  struct.success = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map78.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _key79;
+                  @org.apache.thrift.annotation.Nullable java.lang.String _val80;
+                  for (int _i81 = 0; _i81 < _map78.size; ++_i81)
                   {
-                    _key85 = iprot.readString();
-                    _val86 = iprot.readString();
-                    struct.success.put(_key85, _val86);
+                    _key79 = iprot.readString();
+                    _val80 = iprot.readString();
+                    struct.success.put(_key79, _val80);
                   }
                   iprot.readMapEnd();
                 }
@@ -35023,10 +33122,10 @@
           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())
+            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter82 : struct.success.entrySet())
             {
-              oprot.writeString(_iter88.getKey());
-              oprot.writeString(_iter88.getValue());
+              oprot.writeString(_iter82.getKey());
+              oprot.writeString(_iter82.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -35066,10 +33165,10 @@
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter89 : struct.success.entrySet())
+            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter83 : struct.success.entrySet())
             {
-              oprot.writeString(_iter89.getKey());
-              oprot.writeString(_iter89.getValue());
+              oprot.writeString(_iter83.getKey());
+              oprot.writeString(_iter83.getValue());
             }
           }
         }
@@ -35084,15 +33183,15 @@
         java.util.BitSet incoming = iprot.readBitSet(2);
         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)
+            org.apache.thrift.protocol.TMap _map84 = 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*_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)
             {
-              _key91 = iprot.readString();
-              _val92 = iprot.readString();
-              struct.success.put(_key91, _val92);
+              _key85 = iprot.readString();
+              _val86 = iprot.readString();
+              struct.success.put(_key85, _val86);
             }
           }
           struct.setSuccessIsSet(true);
@@ -35120,7 +33219,7 @@
     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 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. */
@@ -35193,7 +33292,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, 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);
@@ -35204,7 +33303,7 @@
     }
 
     public getVersionedSystemProperties_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials)
     {
       this();
@@ -35217,7 +33316,7 @@
      */
     public getVersionedSystemProperties_args(getVersionedSystemProperties_args other) {
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -35236,11 +33335,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public getVersionedSystemProperties_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public getVersionedSystemProperties_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -35292,7 +33391,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((TInfo)value);
         }
         break;
 
@@ -35506,7 +33605,7 @@
           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 = new TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -35587,7 +33686,7 @@
         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 = new TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -36105,7 +34204,7 @@
     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 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
 
@@ -36182,7 +34281,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, 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, 
@@ -36195,7 +34294,7 @@
     }
 
     public getTableConfiguration_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
       java.lang.String tableName)
     {
@@ -36210,7 +34309,7 @@
      */
     public getTableConfiguration_args(getTableConfiguration_args other) {
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -36233,11 +34332,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public getTableConfiguration_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public getTableConfiguration_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -36314,7 +34413,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((TInfo)value);
         }
         break;
 
@@ -36572,7 +34671,7 @@
           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 = new TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -36672,7 +34771,7 @@
         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 = new TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -37179,15 +35278,15 @@
             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)
+                  org.apache.thrift.protocol.TMap _map88 = iprot.readMapBegin();
+                  struct.success = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map88.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _key89;
+                  @org.apache.thrift.annotation.Nullable java.lang.String _val90;
+                  for (int _i91 = 0; _i91 < _map88.size; ++_i91)
                   {
-                    _key95 = iprot.readString();
-                    _val96 = iprot.readString();
-                    struct.success.put(_key95, _val96);
+                    _key89 = iprot.readString();
+                    _val90 = iprot.readString();
+                    struct.success.put(_key89, _val90);
                   }
                   iprot.readMapEnd();
                 }
@@ -37234,10 +35333,10 @@
           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())
+            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter92 : struct.success.entrySet())
             {
-              oprot.writeString(_iter98.getKey());
-              oprot.writeString(_iter98.getValue());
+              oprot.writeString(_iter92.getKey());
+              oprot.writeString(_iter92.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -37285,10 +35384,10 @@
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter99 : struct.success.entrySet())
+            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter93 : struct.success.entrySet())
             {
-              oprot.writeString(_iter99.getKey());
-              oprot.writeString(_iter99.getValue());
+              oprot.writeString(_iter93.getKey());
+              oprot.writeString(_iter93.getValue());
             }
           }
         }
@@ -37306,15 +35405,15 @@
         java.util.BitSet incoming = iprot.readBitSet(3);
         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)
+            org.apache.thrift.protocol.TMap _map94 = 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*_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)
             {
-              _key101 = iprot.readString();
-              _val102 = iprot.readString();
-              struct.success.put(_key101, _val102);
+              _key95 = iprot.readString();
+              _val96 = iprot.readString();
+              struct.success.put(_key95, _val96);
             }
           }
           struct.setSuccessIsSet(true);
@@ -37348,7 +35447,7 @@
     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 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
 
@@ -37425,7 +35524,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, 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, 
@@ -37438,7 +35537,7 @@
     }
 
     public getTableProperties_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
       java.lang.String tableName)
     {
@@ -37453,7 +35552,7 @@
      */
     public getTableProperties_args(getTableProperties_args other) {
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -37476,11 +35575,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public getTableProperties_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public getTableProperties_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -37557,7 +35656,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((TInfo)value);
         }
         break;
 
@@ -37815,7 +35914,7 @@
           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 = new TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -37915,7 +36014,7 @@
         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 = new TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -38422,15 +36521,15 @@
             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)
+                  org.apache.thrift.protocol.TMap _map98 = iprot.readMapBegin();
+                  struct.success = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map98.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _key99;
+                  @org.apache.thrift.annotation.Nullable java.lang.String _val100;
+                  for (int _i101 = 0; _i101 < _map98.size; ++_i101)
                   {
-                    _key105 = iprot.readString();
-                    _val106 = iprot.readString();
-                    struct.success.put(_key105, _val106);
+                    _key99 = iprot.readString();
+                    _val100 = iprot.readString();
+                    struct.success.put(_key99, _val100);
                   }
                   iprot.readMapEnd();
                 }
@@ -38477,10 +36576,10 @@
           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())
+            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter102 : struct.success.entrySet())
             {
-              oprot.writeString(_iter108.getKey());
-              oprot.writeString(_iter108.getValue());
+              oprot.writeString(_iter102.getKey());
+              oprot.writeString(_iter102.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -38528,10 +36627,10 @@
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter109 : struct.success.entrySet())
+            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter103 : struct.success.entrySet())
             {
-              oprot.writeString(_iter109.getKey());
-              oprot.writeString(_iter109.getValue());
+              oprot.writeString(_iter103.getKey());
+              oprot.writeString(_iter103.getValue());
             }
           }
         }
@@ -38549,15 +36648,15 @@
         java.util.BitSet incoming = iprot.readBitSet(3);
         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)
+            org.apache.thrift.protocol.TMap _map104 = 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*_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)
             {
-              _key111 = iprot.readString();
-              _val112 = iprot.readString();
-              struct.success.put(_key111, _val112);
+              _key105 = iprot.readString();
+              _val106 = iprot.readString();
+              struct.success.put(_key105, _val106);
             }
           }
           struct.setSuccessIsSet(true);
@@ -38591,7 +36690,7 @@
     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 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
 
@@ -38668,7 +36767,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, 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, 
@@ -38681,7 +36780,7 @@
     }
 
     public getVersionedTableProperties_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
       java.lang.String tableName)
     {
@@ -38696,7 +36795,7 @@
      */
     public getVersionedTableProperties_args(getVersionedTableProperties_args other) {
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -38719,11 +36818,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public getVersionedTableProperties_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public getVersionedTableProperties_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -38800,7 +36899,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((TInfo)value);
         }
         break;
 
@@ -39058,7 +37157,7 @@
           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 = new TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -39158,7 +37257,7 @@
         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 = new TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -39787,7 +37886,7 @@
     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 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 ns; // required
 
@@ -39864,7 +37963,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, 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.NS, new org.apache.thrift.meta_data.FieldMetaData("ns", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -39877,7 +37976,7 @@
     }
 
     public getNamespaceConfiguration_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
       java.lang.String ns)
     {
@@ -39892,7 +37991,7 @@
      */
     public getNamespaceConfiguration_args(getNamespaceConfiguration_args other) {
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -39915,11 +38014,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public getNamespaceConfiguration_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public getNamespaceConfiguration_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -39996,7 +38095,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((TInfo)value);
         }
         break;
 
@@ -40254,7 +38353,7 @@
           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 = new TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -40354,7 +38453,7 @@
         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 = new TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -40861,15 +38960,15 @@
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  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)
+                  org.apache.thrift.protocol.TMap _map108 = iprot.readMapBegin();
+                  struct.success = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map108.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _key109;
+                  @org.apache.thrift.annotation.Nullable java.lang.String _val110;
+                  for (int _i111 = 0; _i111 < _map108.size; ++_i111)
                   {
-                    _key115 = iprot.readString();
-                    _val116 = iprot.readString();
-                    struct.success.put(_key115, _val116);
+                    _key109 = iprot.readString();
+                    _val110 = iprot.readString();
+                    struct.success.put(_key109, _val110);
                   }
                   iprot.readMapEnd();
                 }
@@ -40916,10 +39015,10 @@
           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> _iter118 : struct.success.entrySet())
+            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter112 : struct.success.entrySet())
             {
-              oprot.writeString(_iter118.getKey());
-              oprot.writeString(_iter118.getValue());
+              oprot.writeString(_iter112.getKey());
+              oprot.writeString(_iter112.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -40967,10 +39066,10 @@
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter119 : struct.success.entrySet())
+            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter113 : struct.success.entrySet())
             {
-              oprot.writeString(_iter119.getKey());
-              oprot.writeString(_iter119.getValue());
+              oprot.writeString(_iter113.getKey());
+              oprot.writeString(_iter113.getValue());
             }
           }
         }
@@ -40988,15 +39087,15 @@
         java.util.BitSet incoming = iprot.readBitSet(3);
         if (incoming.get(0)) {
           {
-            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)
+            org.apache.thrift.protocol.TMap _map114 = 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*_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)
             {
-              _key121 = iprot.readString();
-              _val122 = iprot.readString();
-              struct.success.put(_key121, _val122);
+              _key115 = iprot.readString();
+              _val116 = iprot.readString();
+              struct.success.put(_key115, _val116);
             }
           }
           struct.setSuccessIsSet(true);
@@ -41030,7 +39129,7 @@
     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 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 ns; // required
 
@@ -41107,7 +39206,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, 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.NS, new org.apache.thrift.meta_data.FieldMetaData("ns", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -41120,7 +39219,7 @@
     }
 
     public getNamespaceProperties_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
       java.lang.String ns)
     {
@@ -41135,7 +39234,7 @@
      */
     public getNamespaceProperties_args(getNamespaceProperties_args other) {
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -41158,11 +39257,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public getNamespaceProperties_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public getNamespaceProperties_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -41239,7 +39338,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((TInfo)value);
         }
         break;
 
@@ -41497,7 +39596,7 @@
           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 = new TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -41597,7 +39696,7 @@
         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 = new TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -42104,15 +40203,15 @@
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map124 = iprot.readMapBegin();
-                  struct.success = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map124.size);
-                  @org.apache.thrift.annotation.Nullable java.lang.String _key125;
-                  @org.apache.thrift.annotation.Nullable java.lang.String _val126;
-                  for (int _i127 = 0; _i127 < _map124.size; ++_i127)
+                  org.apache.thrift.protocol.TMap _map118 = iprot.readMapBegin();
+                  struct.success = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map118.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _key119;
+                  @org.apache.thrift.annotation.Nullable java.lang.String _val120;
+                  for (int _i121 = 0; _i121 < _map118.size; ++_i121)
                   {
-                    _key125 = iprot.readString();
-                    _val126 = iprot.readString();
-                    struct.success.put(_key125, _val126);
+                    _key119 = iprot.readString();
+                    _val120 = iprot.readString();
+                    struct.success.put(_key119, _val120);
                   }
                   iprot.readMapEnd();
                 }
@@ -42159,10 +40258,10 @@
           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> _iter128 : struct.success.entrySet())
+            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter122 : struct.success.entrySet())
             {
-              oprot.writeString(_iter128.getKey());
-              oprot.writeString(_iter128.getValue());
+              oprot.writeString(_iter122.getKey());
+              oprot.writeString(_iter122.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -42210,10 +40309,10 @@
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter129 : struct.success.entrySet())
+            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter123 : struct.success.entrySet())
             {
-              oprot.writeString(_iter129.getKey());
-              oprot.writeString(_iter129.getValue());
+              oprot.writeString(_iter123.getKey());
+              oprot.writeString(_iter123.getValue());
             }
           }
         }
@@ -42231,15 +40330,15 @@
         java.util.BitSet incoming = iprot.readBitSet(3);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TMap _map130 = 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*_map130.size);
-            @org.apache.thrift.annotation.Nullable java.lang.String _key131;
-            @org.apache.thrift.annotation.Nullable java.lang.String _val132;
-            for (int _i133 = 0; _i133 < _map130.size; ++_i133)
+            org.apache.thrift.protocol.TMap _map124 = 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*_map124.size);
+            @org.apache.thrift.annotation.Nullable java.lang.String _key125;
+            @org.apache.thrift.annotation.Nullable java.lang.String _val126;
+            for (int _i127 = 0; _i127 < _map124.size; ++_i127)
             {
-              _key131 = iprot.readString();
-              _val132 = iprot.readString();
-              struct.success.put(_key131, _val132);
+              _key125 = iprot.readString();
+              _val126 = iprot.readString();
+              struct.success.put(_key125, _val126);
             }
           }
           struct.setSuccessIsSet(true);
@@ -42273,7 +40372,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getVersionedNamespaceProperties_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getVersionedNamespaceProperties_argsTupleSchemeFactory();
 
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+    public @org.apache.thrift.annotation.Nullable 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 ns; // required
 
@@ -42350,7 +40449,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, 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.NS, new org.apache.thrift.meta_data.FieldMetaData("ns", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -42363,7 +40462,7 @@
     }
 
     public getVersionedNamespaceProperties_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
       java.lang.String ns)
     {
@@ -42378,7 +40477,7 @@
      */
     public getVersionedNamespaceProperties_args(getVersionedNamespaceProperties_args other) {
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -42401,11 +40500,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public getVersionedNamespaceProperties_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public getVersionedNamespaceProperties_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -42482,7 +40581,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((TInfo)value);
         }
         break;
 
@@ -42740,7 +40839,7 @@
           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 = new TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -42840,7 +40939,7 @@
         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 = new TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -43470,7 +41569,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new checkClass_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new checkClass_argsTupleSchemeFactory();
 
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+    public @org.apache.thrift.annotation.Nullable 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 className; // required
     public @org.apache.thrift.annotation.Nullable java.lang.String interfaceMatch; // required
@@ -43551,7 +41650,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, 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.CLASS_NAME, new org.apache.thrift.meta_data.FieldMetaData("className", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -43566,7 +41665,7 @@
     }
 
     public checkClass_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
       java.lang.String className,
       java.lang.String interfaceMatch)
@@ -43583,7 +41682,7 @@
      */
     public checkClass_args(checkClass_args other) {
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -43610,11 +41709,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public checkClass_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public checkClass_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -43716,7 +41815,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((TInfo)value);
         }
         break;
 
@@ -44018,7 +42117,7 @@
           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 = new TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -44137,7 +42236,7 @@
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
-          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+          struct.tinfo = new TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -44550,7 +42649,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new checkTableClass_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new checkTableClass_argsTupleSchemeFactory();
 
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+    public @org.apache.thrift.annotation.Nullable 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 tableId; // required
     public @org.apache.thrift.annotation.Nullable java.lang.String className; // required
@@ -44635,7 +42734,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, 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_ID, new org.apache.thrift.meta_data.FieldMetaData("tableId", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -44652,7 +42751,7 @@
     }
 
     public checkTableClass_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
       java.lang.String tableId,
       java.lang.String className,
@@ -44671,7 +42770,7 @@
      */
     public checkTableClass_args(checkTableClass_args other) {
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -44702,11 +42801,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public checkTableClass_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public checkTableClass_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -44833,7 +42932,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((TInfo)value);
         }
         break;
 
@@ -45179,7 +43278,7 @@
           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 = new TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -45317,7 +43416,7 @@
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet incoming = iprot.readBitSet(5);
         if (incoming.get(0)) {
-          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+          struct.tinfo = new TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -45948,7 +44047,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new checkNamespaceClass_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new checkNamespaceClass_argsTupleSchemeFactory();
 
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+    public @org.apache.thrift.annotation.Nullable 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 namespaceId; // required
     public @org.apache.thrift.annotation.Nullable java.lang.String className; // required
@@ -46033,7 +44132,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, 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.NAMESPACE_ID, new org.apache.thrift.meta_data.FieldMetaData("namespaceId", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -46050,7 +44149,7 @@
     }
 
     public checkNamespaceClass_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
       java.lang.String namespaceId,
       java.lang.String className,
@@ -46069,7 +44168,7 @@
      */
     public checkNamespaceClass_args(checkNamespaceClass_args other) {
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -46100,11 +44199,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public checkNamespaceClass_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public checkNamespaceClass_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -46231,7 +44330,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((TInfo)value);
         }
         break;
 
@@ -46577,7 +44676,7 @@
           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 = new TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -46715,7 +44814,7 @@
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet incoming = iprot.readBitSet(5);
         if (incoming.get(0)) {
-          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+          struct.tinfo = new TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/trace/thrift/TInfo.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/clientImpl/thrift/TInfo.java
similarity index 91%
rename from core/src/main/thrift-gen-java/org/apache/accumulo/core/trace/thrift/TInfo.java
rename to core/src/main/thrift-gen-java/org/apache/accumulo/core/clientImpl/thrift/TInfo.java
index e657138..d5b9581 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/trace/thrift/TInfo.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/clientImpl/thrift/TInfo.java
@@ -22,13 +22,13 @@
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
  */
-package org.apache.accumulo.core.trace.thrift;
+package org.apache.accumulo.core.clientImpl.thrift;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
 public class TInfo implements org.apache.thrift.TBase<TInfo, TInfo._Fields>, java.io.Serializable, Cloneable, Comparable<TInfo> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TInfo");
 
-  private static final org.apache.thrift.protocol.TField HEADERS_FIELD_DESC = new org.apache.thrift.protocol.TField("headers", org.apache.thrift.protocol.TType.MAP, (short)3);
+  private static final org.apache.thrift.protocol.TField HEADERS_FIELD_DESC = new org.apache.thrift.protocol.TField("headers", org.apache.thrift.protocol.TType.MAP, (short)1);
 
   private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new TInfoStandardSchemeFactory();
   private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new TInfoTupleSchemeFactory();
@@ -37,7 +37,7 @@
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-    HEADERS((short)3, "headers");
+    HEADERS((short)1, "headers");
 
     private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
 
@@ -53,7 +53,7 @@
     @org.apache.thrift.annotation.Nullable
     public static _Fields findByThriftId(int fieldId) {
       switch(fieldId) {
-        case 3: // HEADERS
+        case 1: // HEADERS
           return HEADERS;
         default:
           return null;
@@ -344,18 +344,18 @@
           break;
         }
         switch (schemeField.id) {
-          case 3: // HEADERS
+          case 1: // HEADERS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map0 = iprot.readMapBegin();
-                struct.headers = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map0.size);
-                @org.apache.thrift.annotation.Nullable java.lang.String _key1;
-                @org.apache.thrift.annotation.Nullable java.lang.String _val2;
-                for (int _i3 = 0; _i3 < _map0.size; ++_i3)
+                org.apache.thrift.protocol.TMap _map18 = iprot.readMapBegin();
+                struct.headers = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map18.size);
+                @org.apache.thrift.annotation.Nullable java.lang.String _key19;
+                @org.apache.thrift.annotation.Nullable java.lang.String _val20;
+                for (int _i21 = 0; _i21 < _map18.size; ++_i21)
                 {
-                  _key1 = iprot.readString();
-                  _val2 = iprot.readString();
-                  struct.headers.put(_key1, _val2);
+                  _key19 = iprot.readString();
+                  _val20 = iprot.readString();
+                  struct.headers.put(_key19, _val20);
                 }
                 iprot.readMapEnd();
               }
@@ -384,10 +384,10 @@
         oprot.writeFieldBegin(HEADERS_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.headers.size()));
-          for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter4 : struct.headers.entrySet())
+          for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter22 : struct.headers.entrySet())
           {
-            oprot.writeString(_iter4.getKey());
-            oprot.writeString(_iter4.getValue());
+            oprot.writeString(_iter22.getKey());
+            oprot.writeString(_iter22.getValue());
           }
           oprot.writeMapEnd();
         }
@@ -419,10 +419,10 @@
       if (struct.isSetHeaders()) {
         {
           oprot.writeI32(struct.headers.size());
-          for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter5 : struct.headers.entrySet())
+          for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter23 : struct.headers.entrySet())
           {
-            oprot.writeString(_iter5.getKey());
-            oprot.writeString(_iter5.getValue());
+            oprot.writeString(_iter23.getKey());
+            oprot.writeString(_iter23.getValue());
           }
         }
       }
@@ -434,15 +434,15 @@
       java.util.BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TMap _map6 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); 
-          struct.headers = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map6.size);
-          @org.apache.thrift.annotation.Nullable java.lang.String _key7;
-          @org.apache.thrift.annotation.Nullable java.lang.String _val8;
-          for (int _i9 = 0; _i9 < _map6.size; ++_i9)
+          org.apache.thrift.protocol.TMap _map24 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); 
+          struct.headers = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map24.size);
+          @org.apache.thrift.annotation.Nullable java.lang.String _key25;
+          @org.apache.thrift.annotation.Nullable java.lang.String _val26;
+          for (int _i27 = 0; _i27 < _map24.size; ++_i27)
           {
-            _key7 = iprot.readString();
-            _val8 = iprot.readString();
-            struct.headers.put(_key7, _val8);
+            _key25 = iprot.readString();
+            _val26 = iprot.readString();
+            struct.headers.put(_key25, _val26);
           }
         }
         struct.setHeadersIsSet(true);
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/clientImpl/thrift/TableOperationExceptionType.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/clientImpl/thrift/TableOperationExceptionType.java
index 4b8ec29..ba4a397 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/clientImpl/thrift/TableOperationExceptionType.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/clientImpl/thrift/TableOperationExceptionType.java
@@ -30,13 +30,13 @@
   NOTFOUND(1),
   OFFLINE(2),
   BULK_BAD_INPUT_DIRECTORY(3),
-  BULK_BAD_ERROR_DIRECTORY(4),
+  OBSOLETE_BULK_BAD_ERROR_DIRECTORY(4),
   BAD_RANGE(5),
   OTHER(6),
   NAMESPACE_EXISTS(7),
   NAMESPACE_NOTFOUND(8),
   INVALID_NAME(9),
-  BULK_BAD_LOAD_MAPPING(10),
+  OBSOLETE_BULK_BAD_LOAD_MAPPING(10),
   BULK_CONCURRENT_MERGE(11);
 
   private final int value;
@@ -69,7 +69,7 @@
       case 3:
         return BULK_BAD_INPUT_DIRECTORY;
       case 4:
-        return BULK_BAD_ERROR_DIRECTORY;
+        return OBSOLETE_BULK_BAD_ERROR_DIRECTORY;
       case 5:
         return BAD_RANGE;
       case 6:
@@ -81,7 +81,7 @@
       case 9:
         return INVALID_NAME;
       case 10:
-        return BULK_BAD_LOAD_MAPPING;
+        return OBSOLETE_BULK_BAD_LOAD_MAPPING;
       case 11:
         return BULK_CONCURRENT_MERGE;
       default:
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/compaction/thrift/CompactionCoordinatorService.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/compaction/thrift/CompactionCoordinatorService.java
index ac8e15f..7b8af04 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/compaction/thrift/CompactionCoordinatorService.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/compaction/thrift/CompactionCoordinatorService.java
@@ -29,37 +29,37 @@
 
   public interface Iface {
 
-    public void compactionCompleted(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.accumulo.core.tabletserver.thrift.TCompactionStats stats) throws org.apache.thrift.TException;
+    public void compactionCompleted(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.accumulo.core.tabletserver.thrift.TCompactionStats stats) throws org.apache.thrift.TException;
 
-    public TNextCompactionJob getCompactionJob(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String queueName, java.lang.String compactor, java.lang.String externalCompactionId) throws org.apache.thrift.TException;
+    public TNextCompactionJob getCompactionJob(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String queueName, java.lang.String compactor, java.lang.String externalCompactionId) throws org.apache.thrift.TException;
 
-    public void updateCompactionStatus(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, TCompactionStatusUpdate status, long timestamp) throws org.apache.thrift.TException;
+    public void updateCompactionStatus(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, TCompactionStatusUpdate status, long timestamp) throws org.apache.thrift.TException;
 
-    public void compactionFailed(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent) throws org.apache.thrift.TException;
+    public void compactionFailed(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent) throws org.apache.thrift.TException;
 
-    public TExternalCompactionList getRunningCompactions(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException;
+    public TExternalCompactionList getRunningCompactions(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException;
 
-    public TExternalCompactionList getCompletedCompactions(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException;
+    public TExternalCompactionList getCompletedCompactions(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException;
 
-    public void cancel(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId) throws org.apache.thrift.TException;
+    public void cancel(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId) throws org.apache.thrift.TException;
 
   }
 
   public interface AsyncIface {
 
-    public void compactionCompleted(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.accumulo.core.tabletserver.thrift.TCompactionStats stats, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+    public void compactionCompleted(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.accumulo.core.tabletserver.thrift.TCompactionStats stats, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
 
-    public void getCompactionJob(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String queueName, java.lang.String compactor, java.lang.String externalCompactionId, org.apache.thrift.async.AsyncMethodCallback<TNextCompactionJob> resultHandler) throws org.apache.thrift.TException;
+    public void getCompactionJob(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String queueName, java.lang.String compactor, java.lang.String externalCompactionId, org.apache.thrift.async.AsyncMethodCallback<TNextCompactionJob> resultHandler) throws org.apache.thrift.TException;
 
-    public void updateCompactionStatus(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, TCompactionStatusUpdate status, long timestamp, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+    public void updateCompactionStatus(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, TCompactionStatusUpdate status, long timestamp, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
 
-    public void compactionFailed(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+    public void compactionFailed(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
 
-    public void getRunningCompactions(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<TExternalCompactionList> resultHandler) throws org.apache.thrift.TException;
+    public void getRunningCompactions(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<TExternalCompactionList> resultHandler) throws org.apache.thrift.TException;
 
-    public void getCompletedCompactions(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<TExternalCompactionList> resultHandler) throws org.apache.thrift.TException;
+    public void getCompletedCompactions(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<TExternalCompactionList> resultHandler) throws org.apache.thrift.TException;
 
-    public void cancel(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+    public void cancel(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
 
   }
 
@@ -86,13 +86,13 @@
     }
 
     @Override
-    public void compactionCompleted(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.accumulo.core.tabletserver.thrift.TCompactionStats stats) throws org.apache.thrift.TException
+    public void compactionCompleted(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.accumulo.core.tabletserver.thrift.TCompactionStats stats) throws org.apache.thrift.TException
     {
       send_compactionCompleted(tinfo, credentials, externalCompactionId, extent, stats);
       recv_compactionCompleted();
     }
 
-    public void send_compactionCompleted(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.accumulo.core.tabletserver.thrift.TCompactionStats stats) throws org.apache.thrift.TException
+    public void send_compactionCompleted(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.accumulo.core.tabletserver.thrift.TCompactionStats stats) throws org.apache.thrift.TException
     {
       compactionCompleted_args args = new compactionCompleted_args();
       args.setTinfo(tinfo);
@@ -111,13 +111,13 @@
     }
 
     @Override
-    public TNextCompactionJob getCompactionJob(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String queueName, java.lang.String compactor, java.lang.String externalCompactionId) throws org.apache.thrift.TException
+    public TNextCompactionJob getCompactionJob(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String queueName, java.lang.String compactor, java.lang.String externalCompactionId) throws org.apache.thrift.TException
     {
       send_getCompactionJob(tinfo, credentials, queueName, compactor, externalCompactionId);
       return recv_getCompactionJob();
     }
 
-    public void send_getCompactionJob(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String queueName, java.lang.String compactor, java.lang.String externalCompactionId) throws org.apache.thrift.TException
+    public void send_getCompactionJob(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String queueName, java.lang.String compactor, java.lang.String externalCompactionId) throws org.apache.thrift.TException
     {
       getCompactionJob_args args = new getCompactionJob_args();
       args.setTinfo(tinfo);
@@ -139,13 +139,13 @@
     }
 
     @Override
-    public void updateCompactionStatus(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, TCompactionStatusUpdate status, long timestamp) throws org.apache.thrift.TException
+    public void updateCompactionStatus(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, TCompactionStatusUpdate status, long timestamp) throws org.apache.thrift.TException
     {
       send_updateCompactionStatus(tinfo, credentials, externalCompactionId, status, timestamp);
       recv_updateCompactionStatus();
     }
 
-    public void send_updateCompactionStatus(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, TCompactionStatusUpdate status, long timestamp) throws org.apache.thrift.TException
+    public void send_updateCompactionStatus(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, TCompactionStatusUpdate status, long timestamp) throws org.apache.thrift.TException
     {
       updateCompactionStatus_args args = new updateCompactionStatus_args();
       args.setTinfo(tinfo);
@@ -164,13 +164,13 @@
     }
 
     @Override
-    public void compactionFailed(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent) throws org.apache.thrift.TException
+    public void compactionFailed(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent) throws org.apache.thrift.TException
     {
       send_compactionFailed(tinfo, credentials, externalCompactionId, extent);
       recv_compactionFailed();
     }
 
-    public void send_compactionFailed(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent) throws org.apache.thrift.TException
+    public void send_compactionFailed(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent) throws org.apache.thrift.TException
     {
       compactionFailed_args args = new compactionFailed_args();
       args.setTinfo(tinfo);
@@ -188,13 +188,13 @@
     }
 
     @Override
-    public TExternalCompactionList getRunningCompactions(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
+    public TExternalCompactionList getRunningCompactions(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
     {
       send_getRunningCompactions(tinfo, credentials);
       return recv_getRunningCompactions();
     }
 
-    public void send_getRunningCompactions(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
+    public void send_getRunningCompactions(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
     {
       getRunningCompactions_args args = new getRunningCompactions_args();
       args.setTinfo(tinfo);
@@ -213,13 +213,13 @@
     }
 
     @Override
-    public TExternalCompactionList getCompletedCompactions(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
+    public TExternalCompactionList getCompletedCompactions(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
     {
       send_getCompletedCompactions(tinfo, credentials);
       return recv_getCompletedCompactions();
     }
 
-    public void send_getCompletedCompactions(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
+    public void send_getCompletedCompactions(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
     {
       getCompletedCompactions_args args = new getCompletedCompactions_args();
       args.setTinfo(tinfo);
@@ -238,13 +238,13 @@
     }
 
     @Override
-    public void cancel(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId) throws org.apache.thrift.TException
+    public void cancel(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId) throws org.apache.thrift.TException
     {
       send_cancel(tinfo, credentials, externalCompactionId);
       recv_cancel();
     }
 
-    public void send_cancel(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId) throws org.apache.thrift.TException
+    public void send_cancel(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId) throws org.apache.thrift.TException
     {
       cancel_args args = new cancel_args();
       args.setTinfo(tinfo);
@@ -280,7 +280,7 @@
     }
 
     @Override
-    public void compactionCompleted(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.accumulo.core.tabletserver.thrift.TCompactionStats stats, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+    public void compactionCompleted(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.accumulo.core.tabletserver.thrift.TCompactionStats stats, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       compactionCompleted_call method_call = new compactionCompleted_call(tinfo, credentials, externalCompactionId, extent, stats, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -288,12 +288,12 @@
     }
 
     public static class compactionCompleted_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
       private java.lang.String externalCompactionId;
       private org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent;
       private org.apache.accumulo.core.tabletserver.thrift.TCompactionStats stats;
-      public compactionCompleted_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.accumulo.core.tabletserver.thrift.TCompactionStats stats, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public compactionCompleted_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.accumulo.core.tabletserver.thrift.TCompactionStats stats, org.apache.thrift.async.AsyncMethodCallback<Void> 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;
@@ -328,7 +328,7 @@
     }
 
     @Override
-    public void getCompactionJob(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String queueName, java.lang.String compactor, java.lang.String externalCompactionId, org.apache.thrift.async.AsyncMethodCallback<TNextCompactionJob> resultHandler) throws org.apache.thrift.TException {
+    public void getCompactionJob(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String queueName, java.lang.String compactor, java.lang.String externalCompactionId, org.apache.thrift.async.AsyncMethodCallback<TNextCompactionJob> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       getCompactionJob_call method_call = new getCompactionJob_call(tinfo, credentials, queueName, compactor, externalCompactionId, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -336,12 +336,12 @@
     }
 
     public static class getCompactionJob_call extends org.apache.thrift.async.TAsyncMethodCall<TNextCompactionJob> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
       private java.lang.String queueName;
       private java.lang.String compactor;
       private java.lang.String externalCompactionId;
-      public getCompactionJob_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String queueName, java.lang.String compactor, java.lang.String externalCompactionId, org.apache.thrift.async.AsyncMethodCallback<TNextCompactionJob> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public getCompactionJob_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String queueName, java.lang.String compactor, java.lang.String externalCompactionId, org.apache.thrift.async.AsyncMethodCallback<TNextCompactionJob> 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;
@@ -375,7 +375,7 @@
     }
 
     @Override
-    public void updateCompactionStatus(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, TCompactionStatusUpdate status, long timestamp, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+    public void updateCompactionStatus(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, TCompactionStatusUpdate status, long timestamp, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       updateCompactionStatus_call method_call = new updateCompactionStatus_call(tinfo, credentials, externalCompactionId, status, timestamp, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -383,12 +383,12 @@
     }
 
     public static class updateCompactionStatus_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
       private java.lang.String externalCompactionId;
       private TCompactionStatusUpdate status;
       private long timestamp;
-      public updateCompactionStatus_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, TCompactionStatusUpdate status, long timestamp, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public updateCompactionStatus_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, TCompactionStatusUpdate status, long timestamp, org.apache.thrift.async.AsyncMethodCallback<Void> 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;
@@ -423,7 +423,7 @@
     }
 
     @Override
-    public void compactionFailed(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+    public void compactionFailed(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       compactionFailed_call method_call = new compactionFailed_call(tinfo, credentials, externalCompactionId, extent, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -431,11 +431,11 @@
     }
 
     public static class compactionFailed_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
       private java.lang.String externalCompactionId;
       private org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent;
-      public compactionFailed_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public compactionFailed_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.thrift.async.AsyncMethodCallback<Void> 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;
@@ -468,7 +468,7 @@
     }
 
     @Override
-    public void getRunningCompactions(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<TExternalCompactionList> resultHandler) throws org.apache.thrift.TException {
+    public void getRunningCompactions(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<TExternalCompactionList> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       getRunningCompactions_call method_call = new getRunningCompactions_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -476,9 +476,9 @@
     }
 
     public static class getRunningCompactions_call extends org.apache.thrift.async.TAsyncMethodCall<TExternalCompactionList> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
-      public getRunningCompactions_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<TExternalCompactionList> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public getRunningCompactions_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<TExternalCompactionList> 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;
@@ -506,7 +506,7 @@
     }
 
     @Override
-    public void getCompletedCompactions(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<TExternalCompactionList> resultHandler) throws org.apache.thrift.TException {
+    public void getCompletedCompactions(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<TExternalCompactionList> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       getCompletedCompactions_call method_call = new getCompletedCompactions_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -514,9 +514,9 @@
     }
 
     public static class getCompletedCompactions_call extends org.apache.thrift.async.TAsyncMethodCall<TExternalCompactionList> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
-      public getCompletedCompactions_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<TExternalCompactionList> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public getCompletedCompactions_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<TExternalCompactionList> 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;
@@ -544,7 +544,7 @@
     }
 
     @Override
-    public void cancel(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+    public void cancel(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       cancel_call method_call = new cancel_call(tinfo, credentials, externalCompactionId, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -552,10 +552,10 @@
     }
 
     public static class cancel_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
       private java.lang.String externalCompactionId;
-      public cancel_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public cancel_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.thrift.async.AsyncMethodCallback<Void> 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;
@@ -1307,7 +1307,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new compactionCompleted_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new compactionCompleted_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.clientImpl.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 externalCompactionId; // required
     public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent; // required
@@ -1392,7 +1392,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.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.EXTERNAL_COMPACTION_ID, new org.apache.thrift.meta_data.FieldMetaData("externalCompactionId", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -1409,7 +1409,7 @@
     }
 
     public compactionCompleted_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
       java.lang.String externalCompactionId,
       org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent,
@@ -1428,7 +1428,7 @@
      */
     public compactionCompleted_args(compactionCompleted_args other) {
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -1459,11 +1459,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public compactionCompleted_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public compactionCompleted_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -1590,7 +1590,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
         }
         break;
 
@@ -1942,7 +1942,7 @@
           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 = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -2082,7 +2082,7 @@
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet incoming = iprot.readBitSet(5);
         if (incoming.get(0)) {
-          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+          struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -2391,7 +2391,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getCompactionJob_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getCompactionJob_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.clientImpl.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 queueName; // required
     public @org.apache.thrift.annotation.Nullable java.lang.String compactor; // required
@@ -2476,7 +2476,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.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.QUEUE_NAME, new org.apache.thrift.meta_data.FieldMetaData("queueName", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -2493,7 +2493,7 @@
     }
 
     public getCompactionJob_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
       java.lang.String queueName,
       java.lang.String compactor,
@@ -2512,7 +2512,7 @@
      */
     public getCompactionJob_args(getCompactionJob_args other) {
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -2543,11 +2543,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public getCompactionJob_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public getCompactionJob_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -2674,7 +2674,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
         }
         break;
 
@@ -3020,7 +3020,7 @@
           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 = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -3158,7 +3158,7 @@
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet incoming = iprot.readBitSet(5);
         if (incoming.get(0)) {
-          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+          struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -3583,7 +3583,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new updateCompactionStatus_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new updateCompactionStatus_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.clientImpl.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 externalCompactionId; // required
     public @org.apache.thrift.annotation.Nullable TCompactionStatusUpdate status; // required
@@ -3670,7 +3670,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.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.EXTERNAL_COMPACTION_ID, new org.apache.thrift.meta_data.FieldMetaData("externalCompactionId", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -3687,7 +3687,7 @@
     }
 
     public updateCompactionStatus_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
       java.lang.String externalCompactionId,
       TCompactionStatusUpdate status,
@@ -3708,7 +3708,7 @@
     public updateCompactionStatus_args(updateCompactionStatus_args other) {
       __isset_bitfield = other.__isset_bitfield;
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -3738,11 +3738,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public updateCompactionStatus_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public updateCompactionStatus_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -3867,7 +3867,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
         }
         break;
 
@@ -4212,7 +4212,7 @@
           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 = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -4349,7 +4349,7 @@
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet incoming = iprot.readBitSet(5);
         if (incoming.get(0)) {
-          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+          struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -4656,7 +4656,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new compactionFailed_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new compactionFailed_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.clientImpl.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 externalCompactionId; // required
     public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent; // required
@@ -4737,7 +4737,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.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.EXTERNAL_COMPACTION_ID, new org.apache.thrift.meta_data.FieldMetaData("externalCompactionId", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -4752,7 +4752,7 @@
     }
 
     public compactionFailed_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
       java.lang.String externalCompactionId,
       org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent)
@@ -4769,7 +4769,7 @@
      */
     public compactionFailed_args(compactionFailed_args other) {
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -4796,11 +4796,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public compactionFailed_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public compactionFailed_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -4902,7 +4902,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
         }
         break;
 
@@ -5207,7 +5207,7 @@
           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 = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -5327,7 +5327,7 @@
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
-          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+          struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -5628,7 +5628,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getRunningCompactions_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getRunningCompactions_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.clientImpl.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. */
@@ -5701,7 +5701,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.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);
@@ -5712,7 +5712,7 @@
     }
 
     public getRunningCompactions_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials)
     {
       this();
@@ -5725,7 +5725,7 @@
      */
     public getRunningCompactions_args(getRunningCompactions_args other) {
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -5744,11 +5744,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public getRunningCompactions_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public getRunningCompactions_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -5800,7 +5800,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
         }
         break;
 
@@ -6014,7 +6014,7 @@
           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 = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -6095,7 +6095,7 @@
         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 = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -6505,7 +6505,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getCompletedCompactions_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getCompletedCompactions_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.clientImpl.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. */
@@ -6578,7 +6578,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.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);
@@ -6589,7 +6589,7 @@
     }
 
     public getCompletedCompactions_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials)
     {
       this();
@@ -6602,7 +6602,7 @@
      */
     public getCompletedCompactions_args(getCompletedCompactions_args other) {
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -6621,11 +6621,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public getCompletedCompactions_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public getCompletedCompactions_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -6677,7 +6677,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
         }
         break;
 
@@ -6891,7 +6891,7 @@
           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 = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -6972,7 +6972,7 @@
         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 = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -7383,7 +7383,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new cancel_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new cancel_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.clientImpl.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 externalCompactionId; // required
 
@@ -7460,7 +7460,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.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.EXTERNAL_COMPACTION_ID, new org.apache.thrift.meta_data.FieldMetaData("externalCompactionId", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -7473,7 +7473,7 @@
     }
 
     public cancel_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
       java.lang.String externalCompactionId)
     {
@@ -7488,7 +7488,7 @@
      */
     public cancel_args(cancel_args other) {
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -7511,11 +7511,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public cancel_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public cancel_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -7592,7 +7592,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
         }
         break;
 
@@ -7850,7 +7850,7 @@
           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 = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -7950,7 +7950,7 @@
         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 = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/compaction/thrift/CompactorService.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/compaction/thrift/CompactorService.java
index 90996c5..d7224ae 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/compaction/thrift/CompactorService.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/compaction/thrift/CompactorService.java
@@ -29,25 +29,25 @@
 
   public interface Iface {
 
-    public org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob getRunningCompaction(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException;
+    public org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob getRunningCompaction(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException;
 
-    public java.lang.String getRunningCompactionId(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException;
+    public java.lang.String getRunningCompactionId(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException;
 
-    public java.util.List<org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction> getActiveCompactions(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException;
+    public java.util.List<org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction> getActiveCompactions(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException;
 
-    public void cancel(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId) throws org.apache.thrift.TException;
+    public void cancel(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId) throws org.apache.thrift.TException;
 
   }
 
   public interface AsyncIface {
 
-    public void getRunningCompaction(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob> resultHandler) throws org.apache.thrift.TException;
+    public void getRunningCompaction(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob> resultHandler) throws org.apache.thrift.TException;
 
-    public void getRunningCompactionId(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler) throws org.apache.thrift.TException;
+    public void getRunningCompactionId(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler) throws org.apache.thrift.TException;
 
-    public void getActiveCompactions(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.List<org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction>> resultHandler) throws org.apache.thrift.TException;
+    public void getActiveCompactions(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.List<org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction>> resultHandler) throws org.apache.thrift.TException;
 
-    public void cancel(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+    public void cancel(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
 
   }
 
@@ -74,13 +74,13 @@
     }
 
     @Override
-    public org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob getRunningCompaction(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
+    public org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob getRunningCompaction(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
     {
       send_getRunningCompaction(tinfo, credentials);
       return recv_getRunningCompaction();
     }
 
-    public void send_getRunningCompaction(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
+    public void send_getRunningCompaction(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
     {
       getRunningCompaction_args args = new getRunningCompaction_args();
       args.setTinfo(tinfo);
@@ -102,13 +102,13 @@
     }
 
     @Override
-    public java.lang.String getRunningCompactionId(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
+    public java.lang.String getRunningCompactionId(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
     {
       send_getRunningCompactionId(tinfo, credentials);
       return recv_getRunningCompactionId();
     }
 
-    public void send_getRunningCompactionId(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
+    public void send_getRunningCompactionId(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
     {
       getRunningCompactionId_args args = new getRunningCompactionId_args();
       args.setTinfo(tinfo);
@@ -130,13 +130,13 @@
     }
 
     @Override
-    public java.util.List<org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction> getActiveCompactions(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
+    public java.util.List<org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction> getActiveCompactions(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
     {
       send_getActiveCompactions(tinfo, credentials);
       return recv_getActiveCompactions();
     }
 
-    public void send_getActiveCompactions(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
+    public void send_getActiveCompactions(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
     {
       getActiveCompactions_args args = new getActiveCompactions_args();
       args.setTinfo(tinfo);
@@ -158,13 +158,13 @@
     }
 
     @Override
-    public void cancel(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId) throws org.apache.thrift.TException
+    public void cancel(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId) throws org.apache.thrift.TException
     {
       send_cancel(tinfo, credentials, externalCompactionId);
       recv_cancel();
     }
 
-    public void send_cancel(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId) throws org.apache.thrift.TException
+    public void send_cancel(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId) throws org.apache.thrift.TException
     {
       cancel_args args = new cancel_args();
       args.setTinfo(tinfo);
@@ -200,7 +200,7 @@
     }
 
     @Override
-    public void getRunningCompaction(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob> resultHandler) throws org.apache.thrift.TException {
+    public void getRunningCompaction(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       getRunningCompaction_call method_call = new getRunningCompaction_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -208,9 +208,9 @@
     }
 
     public static class getRunningCompaction_call extends org.apache.thrift.async.TAsyncMethodCall<org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
-      public getRunningCompaction_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public getRunningCompaction_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob> 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;
@@ -238,7 +238,7 @@
     }
 
     @Override
-    public void getRunningCompactionId(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler) throws org.apache.thrift.TException {
+    public void getRunningCompactionId(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       getRunningCompactionId_call method_call = new getRunningCompactionId_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -246,9 +246,9 @@
     }
 
     public static class getRunningCompactionId_call extends org.apache.thrift.async.TAsyncMethodCall<java.lang.String> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
-      public getRunningCompactionId_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public getRunningCompactionId_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
         super(client, protocolFactory, transport, resultHandler, false);
         this.tinfo = tinfo;
         this.credentials = credentials;
@@ -276,7 +276,7 @@
     }
 
     @Override
-    public void getActiveCompactions(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.List<org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction>> resultHandler) throws org.apache.thrift.TException {
+    public void getActiveCompactions(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.List<org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction>> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       getActiveCompactions_call method_call = new getActiveCompactions_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -284,9 +284,9 @@
     }
 
     public static class getActiveCompactions_call extends org.apache.thrift.async.TAsyncMethodCall<java.util.List<org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction>> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
-      public getActiveCompactions_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.List<org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction>> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public getActiveCompactions_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.List<org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction>> 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;
@@ -314,7 +314,7 @@
     }
 
     @Override
-    public void cancel(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+    public void cancel(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       cancel_call method_call = new cancel_call(tinfo, credentials, externalCompactionId, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -322,10 +322,10 @@
     }
 
     public static class cancel_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
       private java.lang.String externalCompactionId;
-      public cancel_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public cancel_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.thrift.async.AsyncMethodCallback<Void> 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;
@@ -810,7 +810,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getRunningCompaction_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getRunningCompaction_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.clientImpl.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. */
@@ -883,7 +883,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.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);
@@ -894,7 +894,7 @@
     }
 
     public getRunningCompaction_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials)
     {
       this();
@@ -907,7 +907,7 @@
      */
     public getRunningCompaction_args(getRunningCompaction_args other) {
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -926,11 +926,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public getRunningCompaction_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public getRunningCompaction_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -982,7 +982,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
         }
         break;
 
@@ -1196,7 +1196,7 @@
           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 = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -1277,7 +1277,7 @@
         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 = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -1794,7 +1794,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getRunningCompactionId_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getRunningCompactionId_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.clientImpl.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. */
@@ -1867,7 +1867,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.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);
@@ -1878,7 +1878,7 @@
     }
 
     public getRunningCompactionId_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials)
     {
       this();
@@ -1891,7 +1891,7 @@
      */
     public getRunningCompactionId_args(getRunningCompactionId_args other) {
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -1910,11 +1910,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public getRunningCompactionId_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public getRunningCompactionId_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -1966,7 +1966,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
         }
         break;
 
@@ -2180,7 +2180,7 @@
           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 = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -2261,7 +2261,7 @@
         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 = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -2773,7 +2773,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getActiveCompactions_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getActiveCompactions_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.clientImpl.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. */
@@ -2846,7 +2846,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.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);
@@ -2857,7 +2857,7 @@
     }
 
     public getActiveCompactions_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials)
     {
       this();
@@ -2870,7 +2870,7 @@
      */
     public getActiveCompactions_args(getActiveCompactions_args other) {
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -2889,11 +2889,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public getActiveCompactions_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public getActiveCompactions_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -2945,7 +2945,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
         }
         break;
 
@@ -3159,7 +3159,7 @@
           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 = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -3240,7 +3240,7 @@
         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 = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -3808,7 +3808,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new cancel_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new cancel_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.clientImpl.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 externalCompactionId; // required
 
@@ -3885,7 +3885,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.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.EXTERNAL_COMPACTION_ID, new org.apache.thrift.meta_data.FieldMetaData("externalCompactionId", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -3898,7 +3898,7 @@
     }
 
     public cancel_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
       java.lang.String externalCompactionId)
     {
@@ -3913,7 +3913,7 @@
      */
     public cancel_args(cancel_args other) {
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -3936,11 +3936,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public cancel_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public cancel_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -4017,7 +4017,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
         }
         break;
 
@@ -4275,7 +4275,7 @@
           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 = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -4375,7 +4375,7 @@
         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 = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/gc/thrift/GCMonitorService.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/gc/thrift/GCMonitorService.java
index c9f2809..840a5e5 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/gc/thrift/GCMonitorService.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/gc/thrift/GCMonitorService.java
@@ -29,13 +29,13 @@
 
   public interface Iface {
 
-    public GCStatus getStatus(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException;
+    public GCStatus getStatus(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException;
 
   }
 
   public interface AsyncIface {
 
-    public void getStatus(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<GCStatus> resultHandler) throws org.apache.thrift.TException;
+    public void getStatus(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<GCStatus> resultHandler) throws org.apache.thrift.TException;
 
   }
 
@@ -62,13 +62,13 @@
     }
 
     @Override
-    public GCStatus getStatus(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
+    public GCStatus getStatus(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
     {
       send_getStatus(tinfo, credentials);
       return recv_getStatus();
     }
 
-    public void send_getStatus(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
+    public void send_getStatus(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
     {
       getStatus_args args = new getStatus_args();
       args.setTinfo(tinfo);
@@ -109,7 +109,7 @@
     }
 
     @Override
-    public void getStatus(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<GCStatus> resultHandler) throws org.apache.thrift.TException {
+    public void getStatus(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<GCStatus> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       getStatus_call method_call = new getStatus_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -117,9 +117,9 @@
     }
 
     public static class getStatus_call extends org.apache.thrift.async.TAsyncMethodCall<GCStatus> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
-      public getStatus_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<GCStatus> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public getStatus_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<GCStatus> 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;
@@ -295,7 +295,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getStatus_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getStatus_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.clientImpl.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. */
@@ -368,7 +368,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.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);
@@ -379,7 +379,7 @@
     }
 
     public getStatus_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials)
     {
       this();
@@ -392,7 +392,7 @@
      */
     public getStatus_args(getStatus_args other) {
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -411,11 +411,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public getStatus_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public getStatus_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -467,7 +467,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
         }
         break;
 
@@ -681,7 +681,7 @@
           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 = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -762,7 +762,7 @@
         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 = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/master/thrift/BulkImportState.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/BulkImportState.java
similarity index 97%
rename from core/src/main/thrift-gen-java/org/apache/accumulo/core/master/thrift/BulkImportState.java
rename to core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/BulkImportState.java
index 0e5f17d..3cb402c 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/master/thrift/BulkImportState.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/BulkImportState.java
@@ -22,7 +22,7 @@
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
  */
-package org.apache.accumulo.core.master.thrift;
+package org.apache.accumulo.core.manager.thrift;
 
 
 public enum BulkImportState implements org.apache.thrift.TEnum {
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/master/thrift/BulkImportStatus.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/BulkImportStatus.java
similarity index 98%
rename from core/src/main/thrift-gen-java/org/apache/accumulo/core/master/thrift/BulkImportStatus.java
rename to core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/BulkImportStatus.java
index 5c92bee..64de85b 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/master/thrift/BulkImportStatus.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/BulkImportStatus.java
@@ -22,7 +22,7 @@
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
  */
-package org.apache.accumulo.core.master.thrift;
+package org.apache.accumulo.core.manager.thrift;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
 public class BulkImportStatus implements org.apache.thrift.TBase<BulkImportStatus, BulkImportStatus._Fields>, java.io.Serializable, Cloneable, Comparable<BulkImportStatus> {
@@ -525,7 +525,7 @@
             break;
           case 3: // STATE
             if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
-              struct.state = org.apache.accumulo.core.master.thrift.BulkImportState.findByValue(iprot.readI32());
+              struct.state = org.apache.accumulo.core.manager.thrift.BulkImportState.findByValue(iprot.readI32());
               struct.setStateIsSet(true);
             } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
@@ -613,7 +613,7 @@
         struct.setFilenameIsSet(true);
       }
       if (incoming.get(2)) {
-        struct.state = org.apache.accumulo.core.master.thrift.BulkImportState.findByValue(iprot.readI32());
+        struct.state = org.apache.accumulo.core.manager.thrift.BulkImportState.findByValue(iprot.readI32());
         struct.setStateIsSet(true);
       }
     }
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/master/thrift/Compacting.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/Compacting.java
similarity index 99%
rename from core/src/main/thrift-gen-java/org/apache/accumulo/core/master/thrift/Compacting.java
rename to core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/Compacting.java
index 7e67064..d21cff5 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/master/thrift/Compacting.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/Compacting.java
@@ -22,7 +22,7 @@
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
  */
-package org.apache.accumulo.core.master.thrift;
+package org.apache.accumulo.core.manager.thrift;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
 public class Compacting implements org.apache.thrift.TBase<Compacting, Compacting._Fields>, java.io.Serializable, Cloneable, Comparable<Compacting> {
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/FateOperation.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/FateOperation.java
index f803d85..a93eb39 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/FateOperation.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/FateOperation.java
@@ -34,7 +34,7 @@
   TABLE_OFFLINE(5),
   TABLE_MERGE(6),
   TABLE_DELETE_RANGE(7),
-  TABLE_BULK_IMPORT(8),
+  OBSOLETE_TABLE_BULK_IMPORT(8),
   TABLE_COMPACT(9),
   TABLE_IMPORT(10),
   TABLE_EXPORT(11),
@@ -82,7 +82,7 @@
       case 7:
         return TABLE_DELETE_RANGE;
       case 8:
-        return TABLE_BULK_IMPORT;
+        return OBSOLETE_TABLE_BULK_IMPORT;
       case 9:
         return TABLE_COMPACT;
       case 10:
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/FateService.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/FateService.java
index c9c0648..89becfd 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/FateService.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/FateService.java
@@ -29,29 +29,29 @@
 
   public interface Iface {
 
-    public long beginFateOperation(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
+    public long beginFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
 
-    public void executeFateOperation(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, FateOperation op, java.util.List<java.nio.ByteBuffer> arguments, java.util.Map<java.lang.String,java.lang.String> options, boolean autoClean) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
+    public void executeFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, FateOperation op, java.util.List<java.nio.ByteBuffer> arguments, java.util.Map<java.lang.String,java.lang.String> options, boolean autoClean) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
 
-    public java.lang.String waitForFateOperation(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
+    public java.lang.String waitForFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
 
-    public void finishFateOperation(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
+    public void finishFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
 
-    public boolean cancelFateOperation(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
+    public boolean cancelFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
 
   }
 
   public interface AsyncIface {
 
-    public void beginFateOperation(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler) throws org.apache.thrift.TException;
+    public void beginFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler) throws org.apache.thrift.TException;
 
-    public void executeFateOperation(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, FateOperation op, java.util.List<java.nio.ByteBuffer> arguments, java.util.Map<java.lang.String,java.lang.String> options, boolean autoClean, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+    public void executeFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, FateOperation op, java.util.List<java.nio.ByteBuffer> arguments, java.util.Map<java.lang.String,java.lang.String> options, boolean autoClean, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
 
-    public void waitForFateOperation(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler) throws org.apache.thrift.TException;
+    public void waitForFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler) throws org.apache.thrift.TException;
 
-    public void finishFateOperation(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+    public void finishFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
 
-    public void cancelFateOperation(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException;
+    public void cancelFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException;
 
   }
 
@@ -78,13 +78,13 @@
     }
 
     @Override
-    public long beginFateOperation(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
+    public long beginFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
     {
       send_beginFateOperation(tinfo, credentials);
       return recv_beginFateOperation();
     }
 
-    public void send_beginFateOperation(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
+    public void send_beginFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
     {
       beginFateOperation_args args = new beginFateOperation_args();
       args.setTinfo(tinfo);
@@ -109,13 +109,13 @@
     }
 
     @Override
-    public void executeFateOperation(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, FateOperation op, java.util.List<java.nio.ByteBuffer> arguments, java.util.Map<java.lang.String,java.lang.String> options, boolean autoClean) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
+    public void executeFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, FateOperation op, java.util.List<java.nio.ByteBuffer> arguments, java.util.Map<java.lang.String,java.lang.String> options, boolean autoClean) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
     {
       send_executeFateOperation(tinfo, credentials, opid, op, arguments, options, autoClean);
       recv_executeFateOperation();
     }
 
-    public void send_executeFateOperation(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, FateOperation op, java.util.List<java.nio.ByteBuffer> arguments, java.util.Map<java.lang.String,java.lang.String> options, boolean autoClean) throws org.apache.thrift.TException
+    public void send_executeFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, FateOperation op, java.util.List<java.nio.ByteBuffer> arguments, java.util.Map<java.lang.String,java.lang.String> options, boolean autoClean) throws org.apache.thrift.TException
     {
       executeFateOperation_args args = new executeFateOperation_args();
       args.setTinfo(tinfo);
@@ -145,13 +145,13 @@
     }
 
     @Override
-    public java.lang.String waitForFateOperation(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
+    public java.lang.String waitForFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
     {
       send_waitForFateOperation(tinfo, credentials, opid);
       return recv_waitForFateOperation();
     }
 
-    public void send_waitForFateOperation(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid) throws org.apache.thrift.TException
+    public void send_waitForFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid) throws org.apache.thrift.TException
     {
       waitForFateOperation_args args = new waitForFateOperation_args();
       args.setTinfo(tinfo);
@@ -180,13 +180,13 @@
     }
 
     @Override
-    public void finishFateOperation(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
+    public void finishFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
     {
       send_finishFateOperation(tinfo, credentials, opid);
       recv_finishFateOperation();
     }
 
-    public void send_finishFateOperation(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid) throws org.apache.thrift.TException
+    public void send_finishFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid) throws org.apache.thrift.TException
     {
       finishFateOperation_args args = new finishFateOperation_args();
       args.setTinfo(tinfo);
@@ -209,13 +209,13 @@
     }
 
     @Override
-    public boolean cancelFateOperation(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
+    public boolean cancelFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
     {
       send_cancelFateOperation(tinfo, credentials, opid);
       return recv_cancelFateOperation();
     }
 
-    public void send_cancelFateOperation(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid) throws org.apache.thrift.TException
+    public void send_cancelFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid) throws org.apache.thrift.TException
     {
       cancelFateOperation_args args = new cancelFateOperation_args();
       args.setTinfo(tinfo);
@@ -260,7 +260,7 @@
     }
 
     @Override
-    public void beginFateOperation(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler) throws org.apache.thrift.TException {
+    public void beginFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       beginFateOperation_call method_call = new beginFateOperation_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -268,9 +268,9 @@
     }
 
     public static class beginFateOperation_call extends org.apache.thrift.async.TAsyncMethodCall<java.lang.Long> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
-      public beginFateOperation_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public beginFateOperation_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> 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;
@@ -298,7 +298,7 @@
     }
 
     @Override
-    public void executeFateOperation(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, FateOperation op, java.util.List<java.nio.ByteBuffer> arguments, java.util.Map<java.lang.String,java.lang.String> options, boolean autoClean, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+    public void executeFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, FateOperation op, java.util.List<java.nio.ByteBuffer> arguments, java.util.Map<java.lang.String,java.lang.String> options, boolean autoClean, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       executeFateOperation_call method_call = new executeFateOperation_call(tinfo, credentials, opid, op, arguments, options, autoClean, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -306,14 +306,14 @@
     }
 
     public static class executeFateOperation_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
       private long opid;
       private FateOperation op;
       private java.util.List<java.nio.ByteBuffer> arguments;
       private java.util.Map<java.lang.String,java.lang.String> options;
       private boolean autoClean;
-      public executeFateOperation_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, FateOperation op, java.util.List<java.nio.ByteBuffer> arguments, java.util.Map<java.lang.String,java.lang.String> options, boolean autoClean, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public executeFateOperation_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, FateOperation op, java.util.List<java.nio.ByteBuffer> arguments, java.util.Map<java.lang.String,java.lang.String> options, boolean autoClean, org.apache.thrift.async.AsyncMethodCallback<Void> 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;
@@ -352,7 +352,7 @@
     }
 
     @Override
-    public void waitForFateOperation(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler) throws org.apache.thrift.TException {
+    public void waitForFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       waitForFateOperation_call method_call = new waitForFateOperation_call(tinfo, credentials, opid, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -360,10 +360,10 @@
     }
 
     public static class waitForFateOperation_call extends org.apache.thrift.async.TAsyncMethodCall<java.lang.String> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
       private long opid;
-      public waitForFateOperation_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public waitForFateOperation_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
         super(client, protocolFactory, transport, resultHandler, false);
         this.tinfo = tinfo;
         this.credentials = credentials;
@@ -393,7 +393,7 @@
     }
 
     @Override
-    public void finishFateOperation(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+    public void finishFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       finishFateOperation_call method_call = new finishFateOperation_call(tinfo, credentials, opid, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -401,10 +401,10 @@
     }
 
     public static class finishFateOperation_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
       private long opid;
-      public finishFateOperation_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public finishFateOperation_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<Void> 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;
@@ -435,7 +435,7 @@
     }
 
     @Override
-    public void cancelFateOperation(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
+    public void cancelFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       cancelFateOperation_call method_call = new cancelFateOperation_call(tinfo, credentials, opid, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -443,10 +443,10 @@
     }
 
     public static class cancelFateOperation_call extends org.apache.thrift.async.TAsyncMethodCall<java.lang.Boolean> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
       private long opid;
-      public cancelFateOperation_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public cancelFateOperation_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> 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;
@@ -1088,7 +1088,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new beginFateOperation_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new beginFateOperation_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.clientImpl.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. */
@@ -1161,7 +1161,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.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);
@@ -1172,7 +1172,7 @@
     }
 
     public beginFateOperation_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials)
     {
       this();
@@ -1185,7 +1185,7 @@
      */
     public beginFateOperation_args(beginFateOperation_args other) {
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -1204,11 +1204,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public beginFateOperation_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public beginFateOperation_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -1260,7 +1260,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
         }
         break;
 
@@ -1474,7 +1474,7 @@
           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 = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -1555,7 +1555,7 @@
         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 = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -2176,7 +2176,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new executeFateOperation_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new executeFateOperation_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.clientImpl.thrift.TInfo tinfo; // required
     public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
     public long opid; // required
     /**
@@ -2280,7 +2280,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.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.OPID, new org.apache.thrift.meta_data.FieldMetaData("opid", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -2304,7 +2304,7 @@
     }
 
     public executeFateOperation_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
       long opid,
       FateOperation op,
@@ -2330,7 +2330,7 @@
     public executeFateOperation_args(executeFateOperation_args other) {
       __isset_bitfield = other.__isset_bitfield;
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -2369,11 +2369,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public executeFateOperation_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public executeFateOperation_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -2581,7 +2581,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
         }
         break;
 
@@ -3005,7 +3005,7 @@
           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 = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -3040,13 +3040,13 @@
             case 5: // ARGUMENTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list60 = iprot.readListBegin();
-                  struct.arguments = new java.util.ArrayList<java.nio.ByteBuffer>(_list60.size);
-                  @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem61;
-                  for (int _i62 = 0; _i62 < _list60.size; ++_i62)
+                  org.apache.thrift.protocol.TList _list86 = iprot.readListBegin();
+                  struct.arguments = new java.util.ArrayList<java.nio.ByteBuffer>(_list86.size);
+                  @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem87;
+                  for (int _i88 = 0; _i88 < _list86.size; ++_i88)
                   {
-                    _elem61 = iprot.readBinary();
-                    struct.arguments.add(_elem61);
+                    _elem87 = iprot.readBinary();
+                    struct.arguments.add(_elem87);
                   }
                   iprot.readListEnd();
                 }
@@ -3058,15 +3058,15 @@
             case 6: // OPTIONS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map63 = iprot.readMapBegin();
-                  struct.options = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map63.size);
-                  @org.apache.thrift.annotation.Nullable java.lang.String _key64;
-                  @org.apache.thrift.annotation.Nullable java.lang.String _val65;
-                  for (int _i66 = 0; _i66 < _map63.size; ++_i66)
+                  org.apache.thrift.protocol.TMap _map89 = iprot.readMapBegin();
+                  struct.options = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map89.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _key90;
+                  @org.apache.thrift.annotation.Nullable java.lang.String _val91;
+                  for (int _i92 = 0; _i92 < _map89.size; ++_i92)
                   {
-                    _key64 = iprot.readString();
-                    _val65 = iprot.readString();
-                    struct.options.put(_key64, _val65);
+                    _key90 = iprot.readString();
+                    _val91 = iprot.readString();
+                    struct.options.put(_key90, _val91);
                   }
                   iprot.readMapEnd();
                 }
@@ -3121,9 +3121,9 @@
           oprot.writeFieldBegin(ARGUMENTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.arguments.size()));
-            for (java.nio.ByteBuffer _iter67 : struct.arguments)
+            for (java.nio.ByteBuffer _iter93 : struct.arguments)
             {
-              oprot.writeBinary(_iter67);
+              oprot.writeBinary(_iter93);
             }
             oprot.writeListEnd();
           }
@@ -3133,10 +3133,10 @@
           oprot.writeFieldBegin(OPTIONS_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.options.size()));
-            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter68 : struct.options.entrySet())
+            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter94 : struct.options.entrySet())
             {
-              oprot.writeString(_iter68.getKey());
-              oprot.writeString(_iter68.getValue());
+              oprot.writeString(_iter94.getKey());
+              oprot.writeString(_iter94.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -3201,19 +3201,19 @@
         if (struct.isSetArguments()) {
           {
             oprot.writeI32(struct.arguments.size());
-            for (java.nio.ByteBuffer _iter69 : struct.arguments)
+            for (java.nio.ByteBuffer _iter95 : struct.arguments)
             {
-              oprot.writeBinary(_iter69);
+              oprot.writeBinary(_iter95);
             }
           }
         }
         if (struct.isSetOptions()) {
           {
             oprot.writeI32(struct.options.size());
-            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter70 : struct.options.entrySet())
+            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter96 : struct.options.entrySet())
             {
-              oprot.writeString(_iter70.getKey());
-              oprot.writeString(_iter70.getValue());
+              oprot.writeString(_iter96.getKey());
+              oprot.writeString(_iter96.getValue());
             }
           }
         }
@@ -3227,7 +3227,7 @@
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet incoming = iprot.readBitSet(7);
         if (incoming.get(0)) {
-          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+          struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -3246,28 +3246,28 @@
         }
         if (incoming.get(4)) {
           {
-            org.apache.thrift.protocol.TList _list71 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
-            struct.arguments = 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)
+            org.apache.thrift.protocol.TList _list97 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
+            struct.arguments = new java.util.ArrayList<java.nio.ByteBuffer>(_list97.size);
+            @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem98;
+            for (int _i99 = 0; _i99 < _list97.size; ++_i99)
             {
-              _elem72 = iprot.readBinary();
-              struct.arguments.add(_elem72);
+              _elem98 = iprot.readBinary();
+              struct.arguments.add(_elem98);
             }
           }
           struct.setArgumentsIsSet(true);
         }
         if (incoming.get(5)) {
           {
-            org.apache.thrift.protocol.TMap _map74 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); 
-            struct.options = 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)
+            org.apache.thrift.protocol.TMap _map100 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); 
+            struct.options = 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)
             {
-              _key75 = iprot.readString();
-              _val76 = iprot.readString();
-              struct.options.put(_key75, _val76);
+              _key101 = iprot.readString();
+              _val102 = iprot.readString();
+              struct.options.put(_key101, _val102);
             }
           }
           struct.setOptionsIsSet(true);
@@ -3889,7 +3889,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new waitForFateOperation_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new waitForFateOperation_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.clientImpl.thrift.TInfo tinfo; // required
     public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
     public long opid; // required
 
@@ -3968,7 +3968,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.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.OPID, new org.apache.thrift.meta_data.FieldMetaData("opid", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -3981,7 +3981,7 @@
     }
 
     public waitForFateOperation_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
       long opid)
     {
@@ -3998,7 +3998,7 @@
     public waitForFateOperation_args(waitForFateOperation_args other) {
       __isset_bitfield = other.__isset_bitfield;
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -4020,11 +4020,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public waitForFateOperation_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public waitForFateOperation_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -4099,7 +4099,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
         }
         break;
 
@@ -4353,7 +4353,7 @@
           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 = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -4451,7 +4451,7 @@
         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 = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -5182,7 +5182,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new finishFateOperation_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new finishFateOperation_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.clientImpl.thrift.TInfo tinfo; // required
     public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
     public long opid; // required
 
@@ -5261,7 +5261,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.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.OPID, new org.apache.thrift.meta_data.FieldMetaData("opid", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -5274,7 +5274,7 @@
     }
 
     public finishFateOperation_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
       long opid)
     {
@@ -5291,7 +5291,7 @@
     public finishFateOperation_args(finishFateOperation_args other) {
       __isset_bitfield = other.__isset_bitfield;
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -5313,11 +5313,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public finishFateOperation_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public finishFateOperation_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -5392,7 +5392,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
         }
         break;
 
@@ -5646,7 +5646,7 @@
           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 = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -5744,7 +5744,7 @@
         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 = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -6263,7 +6263,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new cancelFateOperation_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new cancelFateOperation_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.clientImpl.thrift.TInfo tinfo; // required
     public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
     public long opid; // required
 
@@ -6342,7 +6342,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.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.OPID, new org.apache.thrift.meta_data.FieldMetaData("opid", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -6355,7 +6355,7 @@
     }
 
     public cancelFateOperation_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
       long opid)
     {
@@ -6372,7 +6372,7 @@
     public cancelFateOperation_args(cancelFateOperation_args other) {
       __isset_bitfield = other.__isset_bitfield;
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -6394,11 +6394,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public cancelFateOperation_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public cancelFateOperation_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -6473,7 +6473,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
         }
         break;
 
@@ -6727,7 +6727,7 @@
           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 = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -6825,7 +6825,7 @@
         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 = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/ManagerClientService.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/ManagerClientService.java
index 5e17421..b2c82b5 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/ManagerClientService.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/ManagerClientService.java
@@ -29,93 +29,89 @@
 
   public interface Iface {
 
-    public long initiateFlush(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
+    public long initiateFlush(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
 
-    public void waitForFlush(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.nio.ByteBuffer startRow, java.nio.ByteBuffer endRow, long flushID, long maxLoops) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
+    public void waitForFlush(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.nio.ByteBuffer startRow, java.nio.ByteBuffer endRow, long flushID, long maxLoops) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
 
-    public void setTableProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.lang.String property, java.lang.String value) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, ThriftPropertyException, org.apache.thrift.TException;
+    public void setTableProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.lang.String property, java.lang.String value) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, ThriftPropertyException, org.apache.thrift.TException;
 
-    public void modifyTableProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.accumulo.core.clientImpl.thrift.ThriftConcurrentModificationException, ThriftPropertyException, org.apache.thrift.TException;
+    public void modifyTableProperties(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.accumulo.core.clientImpl.thrift.ThriftConcurrentModificationException, ThriftPropertyException, org.apache.thrift.TException;
 
-    public void removeTableProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.lang.String property) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
+    public void removeTableProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.lang.String property) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
 
-    public void setNamespaceProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, java.lang.String property, java.lang.String value) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, ThriftPropertyException, org.apache.thrift.TException;
+    public void setNamespaceProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, java.lang.String property, java.lang.String value) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, ThriftPropertyException, org.apache.thrift.TException;
 
-    public void modifyNamespaceProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.accumulo.core.clientImpl.thrift.ThriftConcurrentModificationException, org.apache.thrift.TException;
+    public void modifyNamespaceProperties(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.accumulo.core.clientImpl.thrift.ThriftConcurrentModificationException, org.apache.thrift.TException;
 
-    public void removeNamespaceProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, java.lang.String property) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
+    public void removeNamespaceProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, java.lang.String property) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
 
-    public void setManagerGoalState(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, ManagerGoalState state) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
+    public void setManagerGoalState(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, ManagerGoalState state) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
 
-    public void shutdown(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, boolean stopTabletServers) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
+    public void shutdown(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, boolean stopTabletServers) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
 
-    public void shutdownTabletServer(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tabletServer, boolean force) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
+    public void shutdownTabletServer(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tabletServer, boolean force) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
 
-    public void setSystemProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String property, java.lang.String value) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, ThriftPropertyException, org.apache.thrift.TException;
+    public void setSystemProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String property, java.lang.String value) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, ThriftPropertyException, org.apache.thrift.TException;
 
-    public void modifySystemProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.accumulo.core.clientImpl.thrift.ThriftConcurrentModificationException, ThriftPropertyException, org.apache.thrift.TException;
+    public void modifySystemProperties(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.accumulo.core.clientImpl.thrift.ThriftConcurrentModificationException, ThriftPropertyException, org.apache.thrift.TException;
 
-    public void removeSystemProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String property) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
+    public void removeSystemProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String property) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
 
-    public ManagerMonitorInfo getManagerStats(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
+    public ManagerMonitorInfo getManagerStats(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
 
-    public void waitForBalance(org.apache.accumulo.core.trace.thrift.TInfo tinfo) throws org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
+    public void waitForBalance(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) throws org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
 
-    public void reportSplitExtent(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String serverName, TabletSplit split) throws org.apache.thrift.TException;
+    public void reportSplitExtent(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String serverName, TabletSplit split) throws org.apache.thrift.TException;
 
-    public void reportTabletStatus(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String serverName, TabletLoadState status, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent tablet) throws org.apache.thrift.TException;
+    public void reportTabletStatus(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String serverName, TabletLoadState status, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent tablet) throws org.apache.thrift.TException;
 
-    public java.util.List<java.lang.String> getActiveTservers(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
+    public java.util.List<java.lang.String> getActiveTservers(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
 
-    public org.apache.accumulo.core.securityImpl.thrift.TDelegationToken getDelegationToken(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.securityImpl.thrift.TDelegationTokenConfig cfg) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
-
-    public boolean drainReplicationTable(org.apache.accumulo.core.trace.thrift.TInfo tfino, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.util.Set<java.lang.String> logsToWatch) throws org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
+    public org.apache.accumulo.core.securityImpl.thrift.TDelegationToken getDelegationToken(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.securityImpl.thrift.TDelegationTokenConfig cfg) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
 
   }
 
   public interface AsyncIface {
 
-    public void initiateFlush(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.lang.Long> resultHandler) throws org.apache.thrift.TException;
+    public void initiateFlush(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler) throws org.apache.thrift.TException;
 
-    public void waitForFlush(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.nio.ByteBuffer startRow, java.nio.ByteBuffer endRow, long flushID, long maxLoops, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+    public void waitForFlush(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.nio.ByteBuffer startRow, java.nio.ByteBuffer endRow, long flushID, long maxLoops, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
 
-    public void setTableProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.lang.String property, java.lang.String value, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+    public void setTableProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.lang.String property, java.lang.String value, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
 
-    public void modifyTableProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+    public void modifyTableProperties(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
 
-    public void removeTableProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.lang.String property, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+    public void removeTableProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.lang.String property, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
 
-    public void setNamespaceProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, java.lang.String property, java.lang.String value, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+    public void setNamespaceProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, java.lang.String property, java.lang.String value, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
 
-    public void modifyNamespaceProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+    public void modifyNamespaceProperties(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
 
-    public void removeNamespaceProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, java.lang.String property, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+    public void removeNamespaceProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, java.lang.String property, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
 
-    public void setManagerGoalState(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, ManagerGoalState state, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+    public void setManagerGoalState(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, ManagerGoalState state, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
 
-    public void shutdown(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, boolean stopTabletServers, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+    public void shutdown(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, boolean stopTabletServers, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
 
-    public void shutdownTabletServer(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tabletServer, boolean force, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+    public void shutdownTabletServer(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tabletServer, boolean force, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
 
-    public void setSystemProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String property, java.lang.String value, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+    public void setSystemProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String property, java.lang.String value, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
 
-    public void modifySystemProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+    public void modifySystemProperties(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
 
-    public void removeSystemProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String property, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+    public void removeSystemProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String property, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
 
-    public void getManagerStats(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<ManagerMonitorInfo> resultHandler) throws org.apache.thrift.TException;
+    public void getManagerStats(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<ManagerMonitorInfo> resultHandler) throws org.apache.thrift.TException;
 
-    public void waitForBalance(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+    public void waitForBalance(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
 
-    public void reportSplitExtent(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String serverName, TabletSplit split, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+    public void reportSplitExtent(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String serverName, TabletSplit split, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
 
-    public void reportTabletStatus(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String serverName, TabletLoadState status, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent tablet, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+    public void reportTabletStatus(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String serverName, TabletLoadState status, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent tablet, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
 
-    public void getActiveTservers(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.List<java.lang.String>> resultHandler) throws org.apache.thrift.TException;
+    public void getActiveTservers(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.List<java.lang.String>> resultHandler) throws org.apache.thrift.TException;
 
-    public void getDelegationToken(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.securityImpl.thrift.TDelegationTokenConfig cfg, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.securityImpl.thrift.TDelegationToken> resultHandler) throws org.apache.thrift.TException;
-
-    public void drainReplicationTable(org.apache.accumulo.core.trace.thrift.TInfo tfino, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.util.Set<java.lang.String> logsToWatch, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException;
+    public void getDelegationToken(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.securityImpl.thrift.TDelegationTokenConfig cfg, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.securityImpl.thrift.TDelegationToken> resultHandler) throws org.apache.thrift.TException;
 
   }
 
@@ -142,13 +138,13 @@
     }
 
     @Override
-    public long initiateFlush(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
+    public long initiateFlush(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
     {
       send_initiateFlush(tinfo, credentials, tableName);
       return recv_initiateFlush();
     }
 
-    public void send_initiateFlush(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
+    public void send_initiateFlush(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName) throws org.apache.thrift.TException
     {
       initiateFlush_args args = new initiateFlush_args();
       args.setTinfo(tinfo);
@@ -177,13 +173,13 @@
     }
 
     @Override
-    public void waitForFlush(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.nio.ByteBuffer startRow, java.nio.ByteBuffer endRow, long flushID, long maxLoops) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
+    public void waitForFlush(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.nio.ByteBuffer startRow, java.nio.ByteBuffer endRow, long flushID, long maxLoops) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
     {
       send_waitForFlush(tinfo, credentials, tableName, startRow, endRow, flushID, maxLoops);
       recv_waitForFlush();
     }
 
-    public void send_waitForFlush(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.nio.ByteBuffer startRow, java.nio.ByteBuffer endRow, long flushID, long maxLoops) throws org.apache.thrift.TException
+    public void send_waitForFlush(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.nio.ByteBuffer startRow, java.nio.ByteBuffer endRow, long flushID, long maxLoops) throws org.apache.thrift.TException
     {
       waitForFlush_args args = new waitForFlush_args();
       args.setTinfo(tinfo);
@@ -213,13 +209,13 @@
     }
 
     @Override
-    public void setTableProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.lang.String property, java.lang.String value) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, ThriftPropertyException, org.apache.thrift.TException
+    public void setTableProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.lang.String property, java.lang.String value) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, ThriftPropertyException, org.apache.thrift.TException
     {
       send_setTableProperty(tinfo, credentials, tableName, property, value);
       recv_setTableProperty();
     }
 
-    public void send_setTableProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.lang.String property, java.lang.String value) throws org.apache.thrift.TException
+    public void send_setTableProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.lang.String property, java.lang.String value) throws org.apache.thrift.TException
     {
       setTableProperty_args args = new setTableProperty_args();
       args.setTinfo(tinfo);
@@ -250,13 +246,13 @@
     }
 
     @Override
-    public void modifyTableProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.accumulo.core.clientImpl.thrift.ThriftConcurrentModificationException, ThriftPropertyException, org.apache.thrift.TException
+    public void modifyTableProperties(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.accumulo.core.clientImpl.thrift.ThriftConcurrentModificationException, ThriftPropertyException, org.apache.thrift.TException
     {
       send_modifyTableProperties(tinfo, credentials, tableName, vProperties);
       recv_modifyTableProperties();
     }
 
-    public void send_modifyTableProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties) throws org.apache.thrift.TException
+    public void send_modifyTableProperties(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties) throws org.apache.thrift.TException
     {
       modifyTableProperties_args args = new modifyTableProperties_args();
       args.setTinfo(tinfo);
@@ -289,13 +285,13 @@
     }
 
     @Override
-    public void removeTableProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.lang.String property) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
+    public void removeTableProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.lang.String property) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
     {
       send_removeTableProperty(tinfo, credentials, tableName, property);
       recv_removeTableProperty();
     }
 
-    public void send_removeTableProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.lang.String property) throws org.apache.thrift.TException
+    public void send_removeTableProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.lang.String property) throws org.apache.thrift.TException
     {
       removeTableProperty_args args = new removeTableProperty_args();
       args.setTinfo(tinfo);
@@ -322,13 +318,13 @@
     }
 
     @Override
-    public void setNamespaceProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, java.lang.String property, java.lang.String value) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, ThriftPropertyException, org.apache.thrift.TException
+    public void setNamespaceProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, java.lang.String property, java.lang.String value) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, ThriftPropertyException, org.apache.thrift.TException
     {
       send_setNamespaceProperty(tinfo, credentials, ns, property, value);
       recv_setNamespaceProperty();
     }
 
-    public void send_setNamespaceProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, java.lang.String property, java.lang.String value) throws org.apache.thrift.TException
+    public void send_setNamespaceProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, java.lang.String property, java.lang.String value) throws org.apache.thrift.TException
     {
       setNamespaceProperty_args args = new setNamespaceProperty_args();
       args.setTinfo(tinfo);
@@ -359,13 +355,13 @@
     }
 
     @Override
-    public void modifyNamespaceProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.accumulo.core.clientImpl.thrift.ThriftConcurrentModificationException, org.apache.thrift.TException
+    public void modifyNamespaceProperties(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.accumulo.core.clientImpl.thrift.ThriftConcurrentModificationException, org.apache.thrift.TException
     {
       send_modifyNamespaceProperties(tinfo, credentials, ns, vProperties);
       recv_modifyNamespaceProperties();
     }
 
-    public void send_modifyNamespaceProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties) throws org.apache.thrift.TException
+    public void send_modifyNamespaceProperties(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties) throws org.apache.thrift.TException
     {
       modifyNamespaceProperties_args args = new modifyNamespaceProperties_args();
       args.setTinfo(tinfo);
@@ -395,13 +391,13 @@
     }
 
     @Override
-    public void removeNamespaceProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, java.lang.String property) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
+    public void removeNamespaceProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, java.lang.String property) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
     {
       send_removeNamespaceProperty(tinfo, credentials, ns, property);
       recv_removeNamespaceProperty();
     }
 
-    public void send_removeNamespaceProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, java.lang.String property) throws org.apache.thrift.TException
+    public void send_removeNamespaceProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, java.lang.String property) throws org.apache.thrift.TException
     {
       removeNamespaceProperty_args args = new removeNamespaceProperty_args();
       args.setTinfo(tinfo);
@@ -428,13 +424,13 @@
     }
 
     @Override
-    public void setManagerGoalState(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, ManagerGoalState state) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
+    public void setManagerGoalState(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, ManagerGoalState state) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
     {
       send_setManagerGoalState(tinfo, credentials, state);
       recv_setManagerGoalState();
     }
 
-    public void send_setManagerGoalState(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, ManagerGoalState state) throws org.apache.thrift.TException
+    public void send_setManagerGoalState(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, ManagerGoalState state) throws org.apache.thrift.TException
     {
       setManagerGoalState_args args = new setManagerGoalState_args();
       args.setTinfo(tinfo);
@@ -457,13 +453,13 @@
     }
 
     @Override
-    public void shutdown(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, boolean stopTabletServers) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
+    public void shutdown(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, boolean stopTabletServers) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
     {
       send_shutdown(tinfo, credentials, stopTabletServers);
       recv_shutdown();
     }
 
-    public void send_shutdown(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, boolean stopTabletServers) throws org.apache.thrift.TException
+    public void send_shutdown(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, boolean stopTabletServers) throws org.apache.thrift.TException
     {
       shutdown_args args = new shutdown_args();
       args.setTinfo(tinfo);
@@ -486,13 +482,13 @@
     }
 
     @Override
-    public void shutdownTabletServer(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tabletServer, boolean force) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
+    public void shutdownTabletServer(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tabletServer, boolean force) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
     {
       send_shutdownTabletServer(tinfo, credentials, tabletServer, force);
       recv_shutdownTabletServer();
     }
 
-    public void send_shutdownTabletServer(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tabletServer, boolean force) throws org.apache.thrift.TException
+    public void send_shutdownTabletServer(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tabletServer, boolean force) throws org.apache.thrift.TException
     {
       shutdownTabletServer_args args = new shutdownTabletServer_args();
       args.setTinfo(tinfo);
@@ -516,13 +512,13 @@
     }
 
     @Override
-    public void setSystemProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String property, java.lang.String value) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, ThriftPropertyException, org.apache.thrift.TException
+    public void setSystemProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String property, java.lang.String value) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, ThriftPropertyException, org.apache.thrift.TException
     {
       send_setSystemProperty(tinfo, credentials, property, value);
       recv_setSystemProperty();
     }
 
-    public void send_setSystemProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String property, java.lang.String value) throws org.apache.thrift.TException
+    public void send_setSystemProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String property, java.lang.String value) throws org.apache.thrift.TException
     {
       setSystemProperty_args args = new setSystemProperty_args();
       args.setTinfo(tinfo);
@@ -549,13 +545,13 @@
     }
 
     @Override
-    public void modifySystemProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.accumulo.core.clientImpl.thrift.ThriftConcurrentModificationException, ThriftPropertyException, org.apache.thrift.TException
+    public void modifySystemProperties(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.accumulo.core.clientImpl.thrift.ThriftConcurrentModificationException, ThriftPropertyException, org.apache.thrift.TException
     {
       send_modifySystemProperties(tinfo, credentials, vProperties);
       recv_modifySystemProperties();
     }
 
-    public void send_modifySystemProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties) throws org.apache.thrift.TException
+    public void send_modifySystemProperties(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties) throws org.apache.thrift.TException
     {
       modifySystemProperties_args args = new modifySystemProperties_args();
       args.setTinfo(tinfo);
@@ -584,13 +580,13 @@
     }
 
     @Override
-    public void removeSystemProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String property) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
+    public void removeSystemProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String property) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
     {
       send_removeSystemProperty(tinfo, credentials, property);
       recv_removeSystemProperty();
     }
 
-    public void send_removeSystemProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String property) throws org.apache.thrift.TException
+    public void send_removeSystemProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String property) throws org.apache.thrift.TException
     {
       removeSystemProperty_args args = new removeSystemProperty_args();
       args.setTinfo(tinfo);
@@ -613,13 +609,13 @@
     }
 
     @Override
-    public ManagerMonitorInfo getManagerStats(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
+    public ManagerMonitorInfo getManagerStats(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
     {
       send_getManagerStats(tinfo, credentials);
       return recv_getManagerStats();
     }
 
-    public void send_getManagerStats(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
+    public void send_getManagerStats(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
     {
       getManagerStats_args args = new getManagerStats_args();
       args.setTinfo(tinfo);
@@ -644,13 +640,13 @@
     }
 
     @Override
-    public void waitForBalance(org.apache.accumulo.core.trace.thrift.TInfo tinfo) throws org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
+    public void waitForBalance(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) throws org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
     {
       send_waitForBalance(tinfo);
       recv_waitForBalance();
     }
 
-    public void send_waitForBalance(org.apache.accumulo.core.trace.thrift.TInfo tinfo) throws org.apache.thrift.TException
+    public void send_waitForBalance(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) throws org.apache.thrift.TException
     {
       waitForBalance_args args = new waitForBalance_args();
       args.setTinfo(tinfo);
@@ -668,12 +664,12 @@
     }
 
     @Override
-    public void reportSplitExtent(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String serverName, TabletSplit split) throws org.apache.thrift.TException
+    public void reportSplitExtent(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String serverName, TabletSplit split) throws org.apache.thrift.TException
     {
       send_reportSplitExtent(tinfo, credentials, serverName, split);
     }
 
-    public void send_reportSplitExtent(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String serverName, TabletSplit split) throws org.apache.thrift.TException
+    public void send_reportSplitExtent(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String serverName, TabletSplit split) throws org.apache.thrift.TException
     {
       reportSplitExtent_args args = new reportSplitExtent_args();
       args.setTinfo(tinfo);
@@ -684,12 +680,12 @@
     }
 
     @Override
-    public void reportTabletStatus(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String serverName, TabletLoadState status, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent tablet) throws org.apache.thrift.TException
+    public void reportTabletStatus(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String serverName, TabletLoadState status, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent tablet) throws org.apache.thrift.TException
     {
       send_reportTabletStatus(tinfo, credentials, serverName, status, tablet);
     }
 
-    public void send_reportTabletStatus(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String serverName, TabletLoadState status, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent tablet) throws org.apache.thrift.TException
+    public void send_reportTabletStatus(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String serverName, TabletLoadState status, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent tablet) throws org.apache.thrift.TException
     {
       reportTabletStatus_args args = new reportTabletStatus_args();
       args.setTinfo(tinfo);
@@ -701,13 +697,13 @@
     }
 
     @Override
-    public java.util.List<java.lang.String> getActiveTservers(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
+    public java.util.List<java.lang.String> getActiveTservers(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
     {
       send_getActiveTservers(tinfo, credentials);
       return recv_getActiveTservers();
     }
 
-    public void send_getActiveTservers(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
+    public void send_getActiveTservers(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
     {
       getActiveTservers_args args = new getActiveTservers_args();
       args.setTinfo(tinfo);
@@ -732,13 +728,13 @@
     }
 
     @Override
-    public org.apache.accumulo.core.securityImpl.thrift.TDelegationToken getDelegationToken(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.securityImpl.thrift.TDelegationTokenConfig cfg) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
+    public org.apache.accumulo.core.securityImpl.thrift.TDelegationToken getDelegationToken(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.securityImpl.thrift.TDelegationTokenConfig cfg) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
     {
       send_getDelegationToken(tinfo, credentials, cfg);
       return recv_getDelegationToken();
     }
 
-    public void send_getDelegationToken(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.securityImpl.thrift.TDelegationTokenConfig cfg) throws org.apache.thrift.TException
+    public void send_getDelegationToken(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.securityImpl.thrift.TDelegationTokenConfig cfg) throws org.apache.thrift.TException
     {
       getDelegationToken_args args = new getDelegationToken_args();
       args.setTinfo(tinfo);
@@ -763,36 +759,6 @@
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getDelegationToken failed: unknown result");
     }
 
-    @Override
-    public boolean drainReplicationTable(org.apache.accumulo.core.trace.thrift.TInfo tfino, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.util.Set<java.lang.String> logsToWatch) throws org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
-    {
-      send_drainReplicationTable(tfino, credentials, tableName, logsToWatch);
-      return recv_drainReplicationTable();
-    }
-
-    public void send_drainReplicationTable(org.apache.accumulo.core.trace.thrift.TInfo tfino, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.util.Set<java.lang.String> logsToWatch) throws org.apache.thrift.TException
-    {
-      drainReplicationTable_args args = new drainReplicationTable_args();
-      args.setTfino(tfino);
-      args.setCredentials(credentials);
-      args.setTableName(tableName);
-      args.setLogsToWatch(logsToWatch);
-      sendBase("drainReplicationTable", args);
-    }
-
-    public boolean recv_drainReplicationTable() throws org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
-    {
-      drainReplicationTable_result result = new drainReplicationTable_result();
-      receiveBase(result, "drainReplicationTable");
-      if (result.isSetSuccess()) {
-        return result.success;
-      }
-      if (result.tnase != null) {
-        throw result.tnase;
-      }
-      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "drainReplicationTable failed: unknown result");
-    }
-
   }
   public static class AsyncClient extends org.apache.thrift.async.TAsyncClient implements AsyncIface {
     public static class Factory implements org.apache.thrift.async.TAsyncClientFactory<AsyncClient> {
@@ -813,7 +779,7 @@
     }
 
     @Override
-    public void initiateFlush(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.lang.Long> resultHandler) throws org.apache.thrift.TException {
+    public void initiateFlush(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       initiateFlush_call method_call = new initiateFlush_call(tinfo, credentials, tableName, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -821,10 +787,10 @@
     }
 
     public static class initiateFlush_call extends org.apache.thrift.async.TAsyncMethodCall<java.lang.Long> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
       private java.lang.String tableName;
-      public initiateFlush_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.lang.Long> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public initiateFlush_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> 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;
@@ -854,7 +820,7 @@
     }
 
     @Override
-    public void waitForFlush(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.nio.ByteBuffer startRow, java.nio.ByteBuffer endRow, long flushID, long maxLoops, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+    public void waitForFlush(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.nio.ByteBuffer startRow, java.nio.ByteBuffer endRow, long flushID, long maxLoops, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       waitForFlush_call method_call = new waitForFlush_call(tinfo, credentials, tableName, startRow, endRow, flushID, maxLoops, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -862,14 +828,14 @@
     }
 
     public static class waitForFlush_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
       private java.lang.String tableName;
       private java.nio.ByteBuffer startRow;
       private java.nio.ByteBuffer endRow;
       private long flushID;
       private long maxLoops;
-      public waitForFlush_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.nio.ByteBuffer startRow, java.nio.ByteBuffer endRow, long flushID, long maxLoops, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public waitForFlush_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.nio.ByteBuffer startRow, java.nio.ByteBuffer endRow, long flushID, long maxLoops, org.apache.thrift.async.AsyncMethodCallback<Void> 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;
@@ -908,7 +874,7 @@
     }
 
     @Override
-    public void setTableProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.lang.String property, java.lang.String value, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+    public void setTableProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.lang.String property, java.lang.String value, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       setTableProperty_call method_call = new setTableProperty_call(tinfo, credentials, tableName, property, value, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -916,12 +882,12 @@
     }
 
     public static class setTableProperty_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
       private java.lang.String tableName;
       private java.lang.String property;
       private java.lang.String value;
-      public setTableProperty_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.lang.String property, java.lang.String value, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public setTableProperty_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.lang.String property, java.lang.String value, org.apache.thrift.async.AsyncMethodCallback<Void> 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;
@@ -956,7 +922,7 @@
     }
 
     @Override
-    public void modifyTableProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+    public void modifyTableProperties(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       modifyTableProperties_call method_call = new modifyTableProperties_call(tinfo, credentials, tableName, vProperties, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -964,11 +930,11 @@
     }
 
     public static class modifyTableProperties_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
       private java.lang.String tableName;
       private org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties;
-      public modifyTableProperties_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public modifyTableProperties_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties, org.apache.thrift.async.AsyncMethodCallback<Void> 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;
@@ -1001,7 +967,7 @@
     }
 
     @Override
-    public void removeTableProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.lang.String property, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+    public void removeTableProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.lang.String property, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       removeTableProperty_call method_call = new removeTableProperty_call(tinfo, credentials, tableName, property, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -1009,11 +975,11 @@
     }
 
     public static class removeTableProperty_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
       private java.lang.String tableName;
       private java.lang.String property;
-      public removeTableProperty_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.lang.String property, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public removeTableProperty_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.lang.String property, org.apache.thrift.async.AsyncMethodCallback<Void> 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;
@@ -1046,7 +1012,7 @@
     }
 
     @Override
-    public void setNamespaceProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, java.lang.String property, java.lang.String value, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+    public void setNamespaceProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, java.lang.String property, java.lang.String value, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       setNamespaceProperty_call method_call = new setNamespaceProperty_call(tinfo, credentials, ns, property, value, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -1054,12 +1020,12 @@
     }
 
     public static class setNamespaceProperty_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
       private java.lang.String ns;
       private java.lang.String property;
       private java.lang.String value;
-      public setNamespaceProperty_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, java.lang.String property, java.lang.String value, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public setNamespaceProperty_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, java.lang.String property, java.lang.String value, org.apache.thrift.async.AsyncMethodCallback<Void> 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;
@@ -1094,7 +1060,7 @@
     }
 
     @Override
-    public void modifyNamespaceProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+    public void modifyNamespaceProperties(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       modifyNamespaceProperties_call method_call = new modifyNamespaceProperties_call(tinfo, credentials, ns, vProperties, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -1102,11 +1068,11 @@
     }
 
     public static class modifyNamespaceProperties_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
       private java.lang.String ns;
       private org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties;
-      public modifyNamespaceProperties_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public modifyNamespaceProperties_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties, org.apache.thrift.async.AsyncMethodCallback<Void> 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;
@@ -1139,7 +1105,7 @@
     }
 
     @Override
-    public void removeNamespaceProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, java.lang.String property, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+    public void removeNamespaceProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, java.lang.String property, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       removeNamespaceProperty_call method_call = new removeNamespaceProperty_call(tinfo, credentials, ns, property, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -1147,11 +1113,11 @@
     }
 
     public static class removeNamespaceProperty_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
       private java.lang.String ns;
       private java.lang.String property;
-      public removeNamespaceProperty_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, java.lang.String property, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public removeNamespaceProperty_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, java.lang.String property, org.apache.thrift.async.AsyncMethodCallback<Void> 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;
@@ -1184,7 +1150,7 @@
     }
 
     @Override
-    public void setManagerGoalState(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, ManagerGoalState state, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+    public void setManagerGoalState(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, ManagerGoalState state, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       setManagerGoalState_call method_call = new setManagerGoalState_call(tinfo, credentials, state, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -1192,10 +1158,10 @@
     }
 
     public static class setManagerGoalState_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
       private ManagerGoalState state;
-      public setManagerGoalState_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, ManagerGoalState state, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public setManagerGoalState_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, ManagerGoalState state, org.apache.thrift.async.AsyncMethodCallback<Void> 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;
@@ -1226,7 +1192,7 @@
     }
 
     @Override
-    public void shutdown(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, boolean stopTabletServers, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+    public void shutdown(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, boolean stopTabletServers, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       shutdown_call method_call = new shutdown_call(tinfo, credentials, stopTabletServers, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -1234,10 +1200,10 @@
     }
 
     public static class shutdown_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
       private boolean stopTabletServers;
-      public shutdown_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, boolean stopTabletServers, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public shutdown_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, boolean stopTabletServers, org.apache.thrift.async.AsyncMethodCallback<Void> 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;
@@ -1268,7 +1234,7 @@
     }
 
     @Override
-    public void shutdownTabletServer(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tabletServer, boolean force, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+    public void shutdownTabletServer(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tabletServer, boolean force, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       shutdownTabletServer_call method_call = new shutdownTabletServer_call(tinfo, credentials, tabletServer, force, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -1276,11 +1242,11 @@
     }
 
     public static class shutdownTabletServer_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
       private java.lang.String tabletServer;
       private boolean force;
-      public shutdownTabletServer_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tabletServer, boolean force, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public shutdownTabletServer_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tabletServer, boolean force, org.apache.thrift.async.AsyncMethodCallback<Void> 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;
@@ -1313,7 +1279,7 @@
     }
 
     @Override
-    public void setSystemProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String property, java.lang.String value, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+    public void setSystemProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String property, java.lang.String value, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       setSystemProperty_call method_call = new setSystemProperty_call(tinfo, credentials, property, value, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -1321,11 +1287,11 @@
     }
 
     public static class setSystemProperty_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
       private java.lang.String property;
       private java.lang.String value;
-      public setSystemProperty_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String property, java.lang.String value, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public setSystemProperty_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String property, java.lang.String value, org.apache.thrift.async.AsyncMethodCallback<Void> 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;
@@ -1358,7 +1324,7 @@
     }
 
     @Override
-    public void modifySystemProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+    public void modifySystemProperties(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       modifySystemProperties_call method_call = new modifySystemProperties_call(tinfo, credentials, vProperties, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -1366,10 +1332,10 @@
     }
 
     public static class modifySystemProperties_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
       private org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties;
-      public modifySystemProperties_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public modifySystemProperties_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties, org.apache.thrift.async.AsyncMethodCallback<Void> 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;
@@ -1400,7 +1366,7 @@
     }
 
     @Override
-    public void removeSystemProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String property, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+    public void removeSystemProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String property, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       removeSystemProperty_call method_call = new removeSystemProperty_call(tinfo, credentials, property, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -1408,10 +1374,10 @@
     }
 
     public static class removeSystemProperty_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
       private java.lang.String property;
-      public removeSystemProperty_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String property, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public removeSystemProperty_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String property, org.apache.thrift.async.AsyncMethodCallback<Void> 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;
@@ -1442,7 +1408,7 @@
     }
 
     @Override
-    public void getManagerStats(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<ManagerMonitorInfo> resultHandler) throws org.apache.thrift.TException {
+    public void getManagerStats(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<ManagerMonitorInfo> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       getManagerStats_call method_call = new getManagerStats_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -1450,9 +1416,9 @@
     }
 
     public static class getManagerStats_call extends org.apache.thrift.async.TAsyncMethodCall<ManagerMonitorInfo> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
-      public getManagerStats_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<ManagerMonitorInfo> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public getManagerStats_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<ManagerMonitorInfo> 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;
@@ -1480,7 +1446,7 @@
     }
 
     @Override
-    public void waitForBalance(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+    public void waitForBalance(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       waitForBalance_call method_call = new waitForBalance_call(tinfo, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -1488,8 +1454,8 @@
     }
 
     public static class waitForBalance_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
-      public waitForBalance_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
+      public waitForBalance_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.thrift.async.AsyncMethodCallback<Void> 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;
       }
@@ -1516,7 +1482,7 @@
     }
 
     @Override
-    public void reportSplitExtent(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String serverName, TabletSplit split, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+    public void reportSplitExtent(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String serverName, TabletSplit split, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       reportSplitExtent_call method_call = new reportSplitExtent_call(tinfo, credentials, serverName, split, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -1524,11 +1490,11 @@
     }
 
     public static class reportSplitExtent_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
       private java.lang.String serverName;
       private TabletSplit split;
-      public reportSplitExtent_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String serverName, TabletSplit split, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public reportSplitExtent_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String serverName, TabletSplit split, org.apache.thrift.async.AsyncMethodCallback<Void> 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, true);
         this.tinfo = tinfo;
         this.credentials = credentials;
@@ -1560,7 +1526,7 @@
     }
 
     @Override
-    public void reportTabletStatus(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String serverName, TabletLoadState status, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent tablet, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+    public void reportTabletStatus(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String serverName, TabletLoadState status, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent tablet, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       reportTabletStatus_call method_call = new reportTabletStatus_call(tinfo, credentials, serverName, status, tablet, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -1568,12 +1534,12 @@
     }
 
     public static class reportTabletStatus_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
       private java.lang.String serverName;
       private TabletLoadState status;
       private org.apache.accumulo.core.dataImpl.thrift.TKeyExtent tablet;
-      public reportTabletStatus_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String serverName, TabletLoadState status, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent tablet, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public reportTabletStatus_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String serverName, TabletLoadState status, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent tablet, org.apache.thrift.async.AsyncMethodCallback<Void> 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, true);
         this.tinfo = tinfo;
         this.credentials = credentials;
@@ -1607,7 +1573,7 @@
     }
 
     @Override
-    public void getActiveTservers(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.List<java.lang.String>> resultHandler) throws org.apache.thrift.TException {
+    public void getActiveTservers(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.List<java.lang.String>> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       getActiveTservers_call method_call = new getActiveTservers_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -1615,9 +1581,9 @@
     }
 
     public static class getActiveTservers_call extends org.apache.thrift.async.TAsyncMethodCall<java.util.List<java.lang.String>> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
-      public getActiveTservers_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.List<java.lang.String>> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public getActiveTservers_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.List<java.lang.String>> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
         super(client, protocolFactory, transport, resultHandler, false);
         this.tinfo = tinfo;
         this.credentials = credentials;
@@ -1645,7 +1611,7 @@
     }
 
     @Override
-    public void getDelegationToken(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.securityImpl.thrift.TDelegationTokenConfig cfg, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.securityImpl.thrift.TDelegationToken> resultHandler) throws org.apache.thrift.TException {
+    public void getDelegationToken(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.securityImpl.thrift.TDelegationTokenConfig cfg, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.securityImpl.thrift.TDelegationToken> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       getDelegationToken_call method_call = new getDelegationToken_call(tinfo, credentials, cfg, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -1653,10 +1619,10 @@
     }
 
     public static class getDelegationToken_call extends org.apache.thrift.async.TAsyncMethodCall<org.apache.accumulo.core.securityImpl.thrift.TDelegationToken> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
       private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
       private org.apache.accumulo.core.securityImpl.thrift.TDelegationTokenConfig cfg;
-      public getDelegationToken_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.securityImpl.thrift.TDelegationTokenConfig cfg, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.securityImpl.thrift.TDelegationToken> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public getDelegationToken_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.securityImpl.thrift.TDelegationTokenConfig cfg, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.securityImpl.thrift.TDelegationToken> 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;
@@ -1685,50 +1651,6 @@
       }
     }
 
-    @Override
-    public void drainReplicationTable(org.apache.accumulo.core.trace.thrift.TInfo tfino, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.util.Set<java.lang.String> logsToWatch, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
-      checkReady();
-      drainReplicationTable_call method_call = new drainReplicationTable_call(tfino, credentials, tableName, logsToWatch, resultHandler, this, ___protocolFactory, ___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
-    }
-
-    public static class drainReplicationTable_call extends org.apache.thrift.async.TAsyncMethodCall<java.lang.Boolean> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tfino;
-      private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
-      private java.lang.String tableName;
-      private java.util.Set<java.lang.String> logsToWatch;
-      public drainReplicationTable_call(org.apache.accumulo.core.trace.thrift.TInfo tfino, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.util.Set<java.lang.String> logsToWatch, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> 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.tfino = tfino;
-        this.credentials = credentials;
-        this.tableName = tableName;
-        this.logsToWatch = logsToWatch;
-      }
-
-      @Override
-      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("drainReplicationTable", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        drainReplicationTable_args args = new drainReplicationTable_args();
-        args.setTfino(tfino);
-        args.setCredentials(credentials);
-        args.setTableName(tableName);
-        args.setLogsToWatch(logsToWatch);
-        args.write(prot);
-        prot.writeMessageEnd();
-      }
-
-      @Override
-      public java.lang.Boolean getResult() throws org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, 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_drainReplicationTable();
-      }
-    }
-
   }
 
   public static class Processor<I extends Iface> extends org.apache.thrift.TBaseProcessor<I> implements org.apache.thrift.TProcessor {
@@ -1762,7 +1684,6 @@
       processMap.put("reportTabletStatus", new reportTabletStatus());
       processMap.put("getActiveTservers", new getActiveTservers());
       processMap.put("getDelegationToken", new getDelegationToken());
-      processMap.put("drainReplicationTable", new drainReplicationTable());
       return processMap;
     }
 
@@ -2463,39 +2384,6 @@
       }
     }
 
-    public static class drainReplicationTable<I extends Iface> extends org.apache.thrift.ProcessFunction<I, drainReplicationTable_args> {
-      public drainReplicationTable() {
-        super("drainReplicationTable");
-      }
-
-      @Override
-      public drainReplicationTable_args getEmptyArgsInstance() {
-        return new drainReplicationTable_args();
-      }
-
-      @Override
-      protected boolean isOneway() {
-        return false;
-      }
-
-      @Override
-      protected boolean rethrowUnhandledExceptions() {
-        return false;
-      }
-
-      @Override
-      public drainReplicationTable_result getResult(I iface, drainReplicationTable_args args) throws org.apache.thrift.TException {
-        drainReplicationTable_result result = new drainReplicationTable_result();
-        try {
-          result.success = iface.drainReplicationTable(args.tfino, args.credentials, args.tableName, args.logsToWatch);
-          result.setSuccessIsSet(true);
-        } catch (org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException tnase) {
-          result.tnase = tnase;
-        }
-        return result;
-      }
-    }
-
   }
 
   public static class AsyncProcessor<I extends AsyncIface> extends org.apache.thrift.TBaseAsyncProcessor<I> {
@@ -2529,7 +2417,6 @@
       processMap.put("reportTabletStatus", new reportTabletStatus());
       processMap.put("getActiveTservers", new getActiveTservers());
       processMap.put("getDelegationToken", new getDelegationToken());
-      processMap.put("drainReplicationTable", new drainReplicationTable());
       return processMap;
     }
 
@@ -4010,78 +3897,6 @@
       }
     }
 
-    public static class drainReplicationTable<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, drainReplicationTable_args, java.lang.Boolean> {
-      public drainReplicationTable() {
-        super("drainReplicationTable");
-      }
-
-      @Override
-      public drainReplicationTable_args getEmptyArgsInstance() {
-        return new drainReplicationTable_args();
-      }
-
-      @Override
-      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>() { 
-          @Override
-          public void onComplete(java.lang.Boolean o) {
-            drainReplicationTable_result result = new drainReplicationTable_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);
-            }
-          }
-          @Override
-          public void onError(java.lang.Exception e) {
-            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
-            org.apache.thrift.TSerializable msg;
-            drainReplicationTable_result result = new drainReplicationTable_result();
-            if (e instanceof org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException) {
-              result.tnase = (org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException) e;
-              result.setTnaseIsSet(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();
-            }
-          }
-        };
-      }
-
-      @Override
-      protected boolean isOneway() {
-        return false;
-      }
-
-      @Override
-      public void start(I iface, drainReplicationTable_args args, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
-        iface.drainReplicationTable(args.tfino, args.credentials, args.tableName, args.logsToWatch,resultHandler);
-      }
-    }
-
   }
 
   @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
@@ -4095,7 +3910,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new initiateFlush_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new initiateFlush_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.clientImpl.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
 
@@ -4172,7 +3987,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.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, 
@@ -4185,7 +4000,7 @@
     }
 
     public initiateFlush_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
       java.lang.String tableName)
     {
@@ -4200,7 +4015,7 @@
      */
     public initiateFlush_args(initiateFlush_args other) {
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -4223,11 +4038,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public initiateFlush_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public initiateFlush_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -4304,7 +4119,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
         }
         break;
 
@@ -4562,7 +4377,7 @@
           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 = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -4662,7 +4477,7 @@
         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 = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -5394,7 +5209,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new waitForFlush_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new waitForFlush_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.clientImpl.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.nio.ByteBuffer startRow; // required
@@ -5490,7 +5305,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.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, 
@@ -5511,7 +5326,7 @@
     }
 
     public waitForFlush_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
       java.lang.String tableName,
       java.nio.ByteBuffer startRow,
@@ -5537,7 +5352,7 @@
     public waitForFlush_args(waitForFlush_args other) {
       __isset_bitfield = other.__isset_bitfield;
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -5574,11 +5389,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public waitForFlush_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public waitForFlush_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -5769,7 +5584,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
         }
         break;
 
@@ -6201,7 +6016,7 @@
           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 = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -6373,7 +6188,7 @@
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet incoming = iprot.readBitSet(7);
         if (incoming.get(0)) {
-          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+          struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -7017,7 +6832,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new setTableProperty_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new setTableProperty_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.clientImpl.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 property; // required
@@ -7102,7 +6917,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.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, 
@@ -7119,7 +6934,7 @@
     }
 
     public setTableProperty_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
       java.lang.String tableName,
       java.lang.String property,
@@ -7138,7 +6953,7 @@
      */
     public setTableProperty_args(setTableProperty_args other) {
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -7169,11 +6984,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public setTableProperty_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public setTableProperty_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -7300,7 +7115,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
         }
         break;
 
@@ -7646,7 +7461,7 @@
           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 = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -7784,7 +7599,7 @@
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet incoming = iprot.readBitSet(5);
         if (incoming.get(0)) {
-          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+          struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -8526,7 +8341,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new modifyTableProperties_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new modifyTableProperties_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.clientImpl.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 org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties; // required
@@ -8607,7 +8422,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.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, 
@@ -8622,7 +8437,7 @@
     }
 
     public modifyTableProperties_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
       java.lang.String tableName,
       org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties)
@@ -8639,7 +8454,7 @@
      */
     public modifyTableProperties_args(modifyTableProperties_args other) {
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -8666,11 +8481,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public modifyTableProperties_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public modifyTableProperties_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -8772,7 +8587,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
         }
         break;
 
@@ -9077,7 +8892,7 @@
           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 = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -9197,7 +9012,7 @@
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
-          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+          struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -10043,7 +9858,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new removeTableProperty_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new removeTableProperty_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.clientImpl.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 property; // required
@@ -10124,7 +9939,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.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, 
@@ -10139,7 +9954,7 @@
     }
 
     public removeTableProperty_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
       java.lang.String tableName,
       java.lang.String property)
@@ -10156,7 +9971,7 @@
      */
     public removeTableProperty_args(removeTableProperty_args other) {
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -10183,11 +9998,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public removeTableProperty_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public removeTableProperty_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -10289,7 +10104,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
         }
         break;
 
@@ -10591,7 +10406,7 @@
           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 = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -10710,7 +10525,7 @@
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
-          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+          struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -11342,7 +11157,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new setNamespaceProperty_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new setNamespaceProperty_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.clientImpl.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 ns; // required
     public @org.apache.thrift.annotation.Nullable java.lang.String property; // required
@@ -11427,7 +11242,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.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.NS, new org.apache.thrift.meta_data.FieldMetaData("ns", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -11444,7 +11259,7 @@
     }
 
     public setNamespaceProperty_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
       java.lang.String ns,
       java.lang.String property,
@@ -11463,7 +11278,7 @@
      */
     public setNamespaceProperty_args(setNamespaceProperty_args other) {
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -11494,11 +11309,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public setNamespaceProperty_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public setNamespaceProperty_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -11625,7 +11440,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
         }
         break;
 
@@ -11971,7 +11786,7 @@
           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 = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -12109,7 +11924,7 @@
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet incoming = iprot.readBitSet(5);
         if (incoming.get(0)) {
-          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+          struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -12851,7 +12666,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new modifyNamespaceProperties_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new modifyNamespaceProperties_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.clientImpl.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 ns; // required
     public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties; // required
@@ -12932,7 +12747,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.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.NS, new org.apache.thrift.meta_data.FieldMetaData("ns", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -12947,7 +12762,7 @@
     }
 
     public modifyNamespaceProperties_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
       java.lang.String ns,
       org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties)
@@ -12964,7 +12779,7 @@
      */
     public modifyNamespaceProperties_args(modifyNamespaceProperties_args other) {
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -12991,11 +12806,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public modifyNamespaceProperties_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public modifyNamespaceProperties_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -13097,7 +12912,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
         }
         break;
 
@@ -13402,7 +13217,7 @@
           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 = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -13522,7 +13337,7 @@
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
-          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+          struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -14261,7 +14076,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new removeNamespaceProperty_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new removeNamespaceProperty_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.clientImpl.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 ns; // required
     public @org.apache.thrift.annotation.Nullable java.lang.String property; // required
@@ -14342,7 +14157,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.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.NS, new org.apache.thrift.meta_data.FieldMetaData("ns", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -14357,7 +14172,7 @@
     }
 
     public removeNamespaceProperty_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
       java.lang.String ns,
       java.lang.String property)
@@ -14374,7 +14189,7 @@
      */
     public removeNamespaceProperty_args(removeNamespaceProperty_args other) {
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -14401,11 +14216,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public removeNamespaceProperty_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public removeNamespaceProperty_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -14507,7 +14322,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
         }
         break;
 
@@ -14809,7 +14624,7 @@
           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 = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -14928,7 +14743,7 @@
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
-          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+          struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -15558,7 +15373,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new setManagerGoalState_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new setManagerGoalState_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.clientImpl.thrift.TInfo tinfo; // required
     public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
     /**
      * 
@@ -15643,7 +15458,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.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.STATE, new org.apache.thrift.meta_data.FieldMetaData("state", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -15656,7 +15471,7 @@
     }
 
     public setManagerGoalState_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
       ManagerGoalState state)
     {
@@ -15671,7 +15486,7 @@
      */
     public setManagerGoalState_args(setManagerGoalState_args other) {
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -15694,11 +15509,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public setManagerGoalState_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public setManagerGoalState_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -15783,7 +15598,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
         }
         break;
 
@@ -16041,7 +15856,7 @@
           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 = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -16141,7 +15956,7 @@
         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 = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -16660,7 +16475,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new shutdown_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new shutdown_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.clientImpl.thrift.TInfo tinfo; // required
     public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
     public boolean stopTabletServers; // required
 
@@ -16739,7 +16554,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.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.STOP_TABLET_SERVERS, new org.apache.thrift.meta_data.FieldMetaData("stopTabletServers", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -16752,7 +16567,7 @@
     }
 
     public shutdown_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
       boolean stopTabletServers)
     {
@@ -16769,7 +16584,7 @@
     public shutdown_args(shutdown_args other) {
       __isset_bitfield = other.__isset_bitfield;
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -16791,11 +16606,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public shutdown_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public shutdown_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -16870,7 +16685,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
         }
         break;
 
@@ -17124,7 +16939,7 @@
           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 = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -17222,7 +17037,7 @@
         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 = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -17742,7 +17557,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new shutdownTabletServer_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new shutdownTabletServer_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.clientImpl.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 tabletServer; // required
     public boolean force; // required
@@ -17825,7 +17640,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.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.TABLET_SERVER, new org.apache.thrift.meta_data.FieldMetaData("tabletServer", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -17840,7 +17655,7 @@
     }
 
     public shutdownTabletServer_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
       java.lang.String tabletServer,
       boolean force)
@@ -17859,7 +17674,7 @@
     public shutdownTabletServer_args(shutdownTabletServer_args other) {
       __isset_bitfield = other.__isset_bitfield;
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -17885,11 +17700,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public shutdownTabletServer_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public shutdownTabletServer_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -17989,7 +17804,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
         }
         break;
 
@@ -18287,7 +18102,7 @@
           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 = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -18404,7 +18219,7 @@
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
-          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+          struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -18928,7 +18743,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new setSystemProperty_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new setSystemProperty_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.clientImpl.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 property; // required
     public @org.apache.thrift.annotation.Nullable java.lang.String value; // required
@@ -19009,7 +18824,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.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.PROPERTY, new org.apache.thrift.meta_data.FieldMetaData("property", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -19024,7 +18839,7 @@
     }
 
     public setSystemProperty_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
       java.lang.String property,
       java.lang.String value)
@@ -19041,7 +18856,7 @@
      */
     public setSystemProperty_args(setSystemProperty_args other) {
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -19068,11 +18883,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public setSystemProperty_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public setSystemProperty_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -19174,7 +18989,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
         }
         break;
 
@@ -19476,7 +19291,7 @@
           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 = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -19595,7 +19410,7 @@
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
-          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+          struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -20225,7 +20040,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new modifySystemProperties_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new modifySystemProperties_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.clientImpl.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 org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties; // required
 
@@ -20302,7 +20117,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.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.V_PROPERTIES, new org.apache.thrift.meta_data.FieldMetaData("vProperties", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -20315,7 +20130,7 @@
     }
 
     public modifySystemProperties_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
       org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties)
     {
@@ -20330,7 +20145,7 @@
      */
     public modifySystemProperties_args(modifySystemProperties_args other) {
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -20353,11 +20168,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public modifySystemProperties_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public modifySystemProperties_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -20434,7 +20249,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
         }
         break;
 
@@ -20695,7 +20510,7 @@
           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 = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -20796,7 +20611,7 @@
         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 = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -21530,7 +21345,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new removeSystemProperty_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new removeSystemProperty_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.clientImpl.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 property; // required
 
@@ -21607,7 +21422,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.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.PROPERTY, new org.apache.thrift.meta_data.FieldMetaData("property", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -21620,7 +21435,7 @@
     }
 
     public removeSystemProperty_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
       java.lang.String property)
     {
@@ -21635,7 +21450,7 @@
      */
     public removeSystemProperty_args(removeSystemProperty_args other) {
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -21658,11 +21473,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public removeSystemProperty_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public removeSystemProperty_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -21739,7 +21554,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
         }
         break;
 
@@ -21997,7 +21812,7 @@
           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 = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -22097,7 +21912,7 @@
         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 = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -22615,7 +22430,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getManagerStats_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getManagerStats_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.clientImpl.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. */
@@ -22688,7 +22503,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.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);
@@ -22699,7 +22514,7 @@
     }
 
     public getManagerStats_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials)
     {
       this();
@@ -22712,7 +22527,7 @@
      */
     public getManagerStats_args(getManagerStats_args other) {
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -22731,11 +22546,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public getManagerStats_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public getManagerStats_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -22787,7 +22602,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
         }
         break;
 
@@ -23001,7 +22816,7 @@
           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 = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -23082,7 +22897,7 @@
         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 = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -23705,7 +23520,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new waitForBalance_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new waitForBalance_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.clientImpl.thrift.TInfo tinfo; // 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 {
@@ -23774,7 +23589,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.TInfo.class)));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(waitForBalance_args.class, metaDataMap);
     }
@@ -23783,7 +23598,7 @@
     }
 
     public waitForBalance_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo)
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo)
     {
       this();
       this.tinfo = tinfo;
@@ -23794,7 +23609,7 @@
      */
     public waitForBalance_args(waitForBalance_args other) {
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
       }
     }
 
@@ -23809,11 +23624,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public waitForBalance_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public waitForBalance_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -23840,7 +23655,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
         }
         break;
 
@@ -24007,7 +23822,7 @@
           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 = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -24068,7 +23883,7 @@
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
-          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+          struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -24472,7 +24287,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new reportSplitExtent_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new reportSplitExtent_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.clientImpl.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 serverName; // required
     public @org.apache.thrift.annotation.Nullable TabletSplit split; // required
@@ -24553,7 +24368,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.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.SERVER_NAME, new org.apache.thrift.meta_data.FieldMetaData("serverName", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -24568,7 +24383,7 @@
     }
 
     public reportSplitExtent_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
       java.lang.String serverName,
       TabletSplit split)
@@ -24585,7 +24400,7 @@
      */
     public reportSplitExtent_args(reportSplitExtent_args other) {
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -24612,11 +24427,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public reportSplitExtent_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public reportSplitExtent_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -24718,7 +24533,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
         }
         break;
 
@@ -25023,7 +24838,7 @@
           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 = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -25143,7 +24958,7 @@
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
-          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+          struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -25182,7 +24997,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new reportTabletStatus_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new reportTabletStatus_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.clientImpl.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 serverName; // required
     /**
@@ -25275,7 +25090,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.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.SERVER_NAME, new org.apache.thrift.meta_data.FieldMetaData("serverName", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -25292,7 +25107,7 @@
     }
 
     public reportTabletStatus_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
       java.lang.String serverName,
       TabletLoadState status,
@@ -25311,7 +25126,7 @@
      */
     public reportTabletStatus_args(reportTabletStatus_args other) {
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -25342,11 +25157,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public reportTabletStatus_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public reportTabletStatus_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -25481,7 +25296,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
         }
         break;
 
@@ -25830,7 +25645,7 @@
           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 = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -25969,7 +25784,7 @@
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet incoming = iprot.readBitSet(5);
         if (incoming.get(0)) {
-          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+          struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -26009,7 +25824,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getActiveTservers_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getActiveTservers_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.clientImpl.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. */
@@ -26082,7 +25897,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.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);
@@ -26093,7 +25908,7 @@
     }
 
     public getActiveTservers_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials)
     {
       this();
@@ -26106,7 +25921,7 @@
      */
     public getActiveTservers_args(getActiveTservers_args other) {
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -26125,11 +25940,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public getActiveTservers_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public getActiveTservers_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -26181,7 +25996,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
         }
         break;
 
@@ -26395,7 +26210,7 @@
           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 = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -26476,7 +26291,7 @@
         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 = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -26983,13 +26798,13 @@
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list78 = iprot.readListBegin();
-                  struct.success = new java.util.ArrayList<java.lang.String>(_list78.size);
-                  @org.apache.thrift.annotation.Nullable java.lang.String _elem79;
-                  for (int _i80 = 0; _i80 < _list78.size; ++_i80)
+                  org.apache.thrift.protocol.TList _list104 = iprot.readListBegin();
+                  struct.success = new java.util.ArrayList<java.lang.String>(_list104.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _elem105;
+                  for (int _i106 = 0; _i106 < _list104.size; ++_i106)
                   {
-                    _elem79 = iprot.readString();
-                    struct.success.add(_elem79);
+                    _elem105 = iprot.readString();
+                    struct.success.add(_elem105);
                   }
                   iprot.readListEnd();
                 }
@@ -27036,9 +26851,9 @@
           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 _iter81 : struct.success)
+            for (java.lang.String _iter107 : struct.success)
             {
-              oprot.writeString(_iter81);
+              oprot.writeString(_iter107);
             }
             oprot.writeListEnd();
           }
@@ -27086,9 +26901,9 @@
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (java.lang.String _iter82 : struct.success)
+            for (java.lang.String _iter108 : struct.success)
             {
-              oprot.writeString(_iter82);
+              oprot.writeString(_iter108);
             }
           }
         }
@@ -27106,13 +26921,13 @@
         java.util.BitSet incoming = iprot.readBitSet(3);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list83 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
-            struct.success = new java.util.ArrayList<java.lang.String>(_list83.size);
-            @org.apache.thrift.annotation.Nullable java.lang.String _elem84;
-            for (int _i85 = 0; _i85 < _list83.size; ++_i85)
+            org.apache.thrift.protocol.TList _list109 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
+            struct.success = new java.util.ArrayList<java.lang.String>(_list109.size);
+            @org.apache.thrift.annotation.Nullable java.lang.String _elem110;
+            for (int _i111 = 0; _i111 < _list109.size; ++_i111)
             {
-              _elem84 = iprot.readString();
-              struct.success.add(_elem84);
+              _elem110 = iprot.readString();
+              struct.success.add(_elem110);
             }
           }
           struct.setSuccessIsSet(true);
@@ -27146,7 +26961,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getDelegationToken_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getDelegationToken_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.clientImpl.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 org.apache.accumulo.core.securityImpl.thrift.TDelegationTokenConfig cfg; // required
 
@@ -27223,7 +27038,7 @@
     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)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.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.CFG, new org.apache.thrift.meta_data.FieldMetaData("cfg", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -27236,7 +27051,7 @@
     }
 
     public getDelegationToken_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
       org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
       org.apache.accumulo.core.securityImpl.thrift.TDelegationTokenConfig cfg)
     {
@@ -27251,7 +27066,7 @@
      */
     public getDelegationToken_args(getDelegationToken_args other) {
       if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+        this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
       }
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -27274,11 +27089,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
       return this.tinfo;
     }
 
-    public getDelegationToken_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public getDelegationToken_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -27355,7 +27170,7 @@
         if (value == null) {
           unsetTinfo();
         } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+          setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
         }
         break;
 
@@ -27616,7 +27431,7 @@
           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 = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
                 struct.tinfo.read(iprot);
                 struct.setTinfoIsSet(true);
               } else { 
@@ -27717,7 +27532,7 @@
         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 = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
           struct.tinfo.read(iprot);
           struct.setTinfoIsSet(true);
         }
@@ -28336,1241 +28151,5 @@
     }
   }
 
-  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-  public static class drainReplicationTable_args implements org.apache.thrift.TBase<drainReplicationTable_args, drainReplicationTable_args._Fields>, java.io.Serializable, Cloneable, Comparable<drainReplicationTable_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("drainReplicationTable_args");
-
-    private static final org.apache.thrift.protocol.TField TFINO_FIELD_DESC = new org.apache.thrift.protocol.TField("tfino", 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 TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)3);
-    private static final org.apache.thrift.protocol.TField LOGS_TO_WATCH_FIELD_DESC = new org.apache.thrift.protocol.TField("logsToWatch", org.apache.thrift.protocol.TType.SET, (short)4);
-
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new drainReplicationTable_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new drainReplicationTable_argsTupleSchemeFactory();
-
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tfino; // 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.util.Set<java.lang.String> logsToWatch; // 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 {
-      TFINO((short)1, "tfino"),
-      CREDENTIALS((short)2, "credentials"),
-      TABLE_NAME((short)3, "tableName"),
-      LOGS_TO_WATCH((short)4, "logsToWatch");
-
-      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: // TFINO
-            return TFINO;
-          case 2: // CREDENTIALS
-            return CREDENTIALS;
-          case 3: // TABLE_NAME
-            return TABLE_NAME;
-          case 4: // LOGS_TO_WATCH
-            return LOGS_TO_WATCH;
-          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;
-      }
-
-      @Override
-      public short getThriftFieldId() {
-        return _thriftId;
-      }
-
-      @Override
-      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.TFINO, new org.apache.thrift.meta_data.FieldMetaData("tfino", 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)));
-      tmpMap.put(_Fields.LOGS_TO_WATCH, new org.apache.thrift.meta_data.FieldMetaData("logsToWatch", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.SetMetaData(org.apache.thrift.protocol.TType.SET, 
-              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(drainReplicationTable_args.class, metaDataMap);
-    }
-
-    public drainReplicationTable_args() {
-    }
-
-    public drainReplicationTable_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tfino,
-      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
-      java.lang.String tableName,
-      java.util.Set<java.lang.String> logsToWatch)
-    {
-      this();
-      this.tfino = tfino;
-      this.credentials = credentials;
-      this.tableName = tableName;
-      this.logsToWatch = logsToWatch;
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public drainReplicationTable_args(drainReplicationTable_args other) {
-      if (other.isSetTfino()) {
-        this.tfino = new org.apache.accumulo.core.trace.thrift.TInfo(other.tfino);
-      }
-      if (other.isSetCredentials()) {
-        this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
-      }
-      if (other.isSetTableName()) {
-        this.tableName = other.tableName;
-      }
-      if (other.isSetLogsToWatch()) {
-        java.util.Set<java.lang.String> __this__logsToWatch = new java.util.HashSet<java.lang.String>(other.logsToWatch);
-        this.logsToWatch = __this__logsToWatch;
-      }
-    }
-
-    @Override
-    public drainReplicationTable_args deepCopy() {
-      return new drainReplicationTable_args(this);
-    }
-
-    @Override
-    public void clear() {
-      this.tfino = null;
-      this.credentials = null;
-      this.tableName = null;
-      this.logsToWatch = null;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTfino() {
-      return this.tfino;
-    }
-
-    public drainReplicationTable_args setTfino(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tfino) {
-      this.tfino = tfino;
-      return this;
-    }
-
-    public void unsetTfino() {
-      this.tfino = null;
-    }
-
-    /** Returns true if field tfino is set (has been assigned a value) and false otherwise */
-    public boolean isSetTfino() {
-      return this.tfino != null;
-    }
-
-    public void setTfinoIsSet(boolean value) {
-      if (!value) {
-        this.tfino = null;
-      }
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.securityImpl.thrift.TCredentials getCredentials() {
-      return this.credentials;
-    }
-
-    public drainReplicationTable_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 drainReplicationTable_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 int getLogsToWatchSize() {
-      return (this.logsToWatch == null) ? 0 : this.logsToWatch.size();
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.util.Iterator<java.lang.String> getLogsToWatchIterator() {
-      return (this.logsToWatch == null) ? null : this.logsToWatch.iterator();
-    }
-
-    public void addToLogsToWatch(java.lang.String elem) {
-      if (this.logsToWatch == null) {
-        this.logsToWatch = new java.util.HashSet<java.lang.String>();
-      }
-      this.logsToWatch.add(elem);
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.util.Set<java.lang.String> getLogsToWatch() {
-      return this.logsToWatch;
-    }
-
-    public drainReplicationTable_args setLogsToWatch(@org.apache.thrift.annotation.Nullable java.util.Set<java.lang.String> logsToWatch) {
-      this.logsToWatch = logsToWatch;
-      return this;
-    }
-
-    public void unsetLogsToWatch() {
-      this.logsToWatch = null;
-    }
-
-    /** Returns true if field logsToWatch is set (has been assigned a value) and false otherwise */
-    public boolean isSetLogsToWatch() {
-      return this.logsToWatch != null;
-    }
-
-    public void setLogsToWatchIsSet(boolean value) {
-      if (!value) {
-        this.logsToWatch = null;
-      }
-    }
-
-    @Override
-    public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
-      switch (field) {
-      case TFINO:
-        if (value == null) {
-          unsetTfino();
-        } else {
-          setTfino((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;
-
-      case LOGS_TO_WATCH:
-        if (value == null) {
-          unsetLogsToWatch();
-        } else {
-          setLogsToWatch((java.util.Set<java.lang.String>)value);
-        }
-        break;
-
-      }
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    @Override
-    public java.lang.Object getFieldValue(_Fields field) {
-      switch (field) {
-      case TFINO:
-        return getTfino();
-
-      case CREDENTIALS:
-        return getCredentials();
-
-      case TABLE_NAME:
-        return getTableName();
-
-      case LOGS_TO_WATCH:
-        return getLogsToWatch();
-
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
-    @Override
-    public boolean isSet(_Fields field) {
-      if (field == null) {
-        throw new java.lang.IllegalArgumentException();
-      }
-
-      switch (field) {
-      case TFINO:
-        return isSetTfino();
-      case CREDENTIALS:
-        return isSetCredentials();
-      case TABLE_NAME:
-        return isSetTableName();
-      case LOGS_TO_WATCH:
-        return isSetLogsToWatch();
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(java.lang.Object that) {
-      if (that instanceof drainReplicationTable_args)
-        return this.equals((drainReplicationTable_args)that);
-      return false;
-    }
-
-    public boolean equals(drainReplicationTable_args that) {
-      if (that == null)
-        return false;
-      if (this == that)
-        return true;
-
-      boolean this_present_tfino = true && this.isSetTfino();
-      boolean that_present_tfino = true && that.isSetTfino();
-      if (this_present_tfino || that_present_tfino) {
-        if (!(this_present_tfino && that_present_tfino))
-          return false;
-        if (!this.tfino.equals(that.tfino))
-          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;
-      }
-
-      boolean this_present_logsToWatch = true && this.isSetLogsToWatch();
-      boolean that_present_logsToWatch = true && that.isSetLogsToWatch();
-      if (this_present_logsToWatch || that_present_logsToWatch) {
-        if (!(this_present_logsToWatch && that_present_logsToWatch))
-          return false;
-        if (!this.logsToWatch.equals(that.logsToWatch))
-          return false;
-      }
-
-      return true;
-    }
-
-    @Override
-    public int hashCode() {
-      int hashCode = 1;
-
-      hashCode = hashCode * 8191 + ((isSetTfino()) ? 131071 : 524287);
-      if (isSetTfino())
-        hashCode = hashCode * 8191 + tfino.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();
-
-      hashCode = hashCode * 8191 + ((isSetLogsToWatch()) ? 131071 : 524287);
-      if (isSetLogsToWatch())
-        hashCode = hashCode * 8191 + logsToWatch.hashCode();
-
-      return hashCode;
-    }
-
-    @Override
-    public int compareTo(drainReplicationTable_args other) {
-      if (!getClass().equals(other.getClass())) {
-        return getClass().getName().compareTo(other.getClass().getName());
-      }
-
-      int lastComparison = 0;
-
-      lastComparison = java.lang.Boolean.compare(isSetTfino(), other.isSetTfino());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetTfino()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tfino, other.tfino);
-        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;
-        }
-      }
-      lastComparison = java.lang.Boolean.compare(isSetLogsToWatch(), other.isSetLogsToWatch());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetLogsToWatch()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.logsToWatch, other.logsToWatch);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      return 0;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    @Override
-    public _Fields fieldForId(int fieldId) {
-      return _Fields.findByThriftId(fieldId);
-    }
-
-    @Override
-    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
-      scheme(iprot).read(iprot, this);
-    }
-
-    @Override
-    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("drainReplicationTable_args(");
-      boolean first = true;
-
-      sb.append("tfino:");
-      if (this.tfino == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.tfino);
-      }
-      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;
-      if (!first) sb.append(", ");
-      sb.append("logsToWatch:");
-      if (this.logsToWatch == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.logsToWatch);
-      }
-      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 (tfino != null) {
-        tfino.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 drainReplicationTable_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public drainReplicationTable_argsStandardScheme getScheme() {
-        return new drainReplicationTable_argsStandardScheme();
-      }
-    }
-
-    private static class drainReplicationTable_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<drainReplicationTable_args> {
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol iprot, drainReplicationTable_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: // TFINO
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.tfino = new org.apache.accumulo.core.trace.thrift.TInfo();
-                struct.tfino.read(iprot);
-                struct.setTfinoIsSet(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;
-            case 3: // 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;
-            case 4: // LOGS_TO_WATCH
-              if (schemeField.type == org.apache.thrift.protocol.TType.SET) {
-                {
-                  org.apache.thrift.protocol.TSet _set86 = iprot.readSetBegin();
-                  struct.logsToWatch = new java.util.HashSet<java.lang.String>(2*_set86.size);
-                  @org.apache.thrift.annotation.Nullable java.lang.String _elem87;
-                  for (int _i88 = 0; _i88 < _set86.size; ++_i88)
-                  {
-                    _elem87 = iprot.readString();
-                    struct.logsToWatch.add(_elem87);
-                  }
-                  iprot.readSetEnd();
-                }
-                struct.setLogsToWatchIsSet(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();
-      }
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol oprot, drainReplicationTable_args struct) throws org.apache.thrift.TException {
-        struct.validate();
-
-        oprot.writeStructBegin(STRUCT_DESC);
-        if (struct.tfino != null) {
-          oprot.writeFieldBegin(TFINO_FIELD_DESC);
-          struct.tfino.write(oprot);
-          oprot.writeFieldEnd();
-        }
-        if (struct.credentials != null) {
-          oprot.writeFieldBegin(CREDENTIALS_FIELD_DESC);
-          struct.credentials.write(oprot);
-          oprot.writeFieldEnd();
-        }
-        if (struct.tableName != null) {
-          oprot.writeFieldBegin(TABLE_NAME_FIELD_DESC);
-          oprot.writeString(struct.tableName);
-          oprot.writeFieldEnd();
-        }
-        if (struct.logsToWatch != null) {
-          oprot.writeFieldBegin(LOGS_TO_WATCH_FIELD_DESC);
-          {
-            oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRING, struct.logsToWatch.size()));
-            for (java.lang.String _iter89 : struct.logsToWatch)
-            {
-              oprot.writeString(_iter89);
-            }
-            oprot.writeSetEnd();
-          }
-          oprot.writeFieldEnd();
-        }
-        oprot.writeFieldStop();
-        oprot.writeStructEnd();
-      }
-
-    }
-
-    private static class drainReplicationTable_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public drainReplicationTable_argsTupleScheme getScheme() {
-        return new drainReplicationTable_argsTupleScheme();
-      }
-    }
-
-    private static class drainReplicationTable_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<drainReplicationTable_args> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, drainReplicationTable_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.isSetTfino()) {
-          optionals.set(0);
-        }
-        if (struct.isSetCredentials()) {
-          optionals.set(1);
-        }
-        if (struct.isSetTableName()) {
-          optionals.set(2);
-        }
-        if (struct.isSetLogsToWatch()) {
-          optionals.set(3);
-        }
-        oprot.writeBitSet(optionals, 4);
-        if (struct.isSetTfino()) {
-          struct.tfino.write(oprot);
-        }
-        if (struct.isSetCredentials()) {
-          struct.credentials.write(oprot);
-        }
-        if (struct.isSetTableName()) {
-          oprot.writeString(struct.tableName);
-        }
-        if (struct.isSetLogsToWatch()) {
-          {
-            oprot.writeI32(struct.logsToWatch.size());
-            for (java.lang.String _iter90 : struct.logsToWatch)
-            {
-              oprot.writeString(_iter90);
-            }
-          }
-        }
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, drainReplicationTable_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(4);
-        if (incoming.get(0)) {
-          struct.tfino = new org.apache.accumulo.core.trace.thrift.TInfo();
-          struct.tfino.read(iprot);
-          struct.setTfinoIsSet(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);
-        }
-        if (incoming.get(3)) {
-          {
-            org.apache.thrift.protocol.TSet _set91 = iprot.readSetBegin(org.apache.thrift.protocol.TType.STRING);
-            struct.logsToWatch = new java.util.HashSet<java.lang.String>(2*_set91.size);
-            @org.apache.thrift.annotation.Nullable java.lang.String _elem92;
-            for (int _i93 = 0; _i93 < _set91.size; ++_i93)
-            {
-              _elem92 = iprot.readString();
-              struct.logsToWatch.add(_elem92);
-            }
-          }
-          struct.setLogsToWatchIsSet(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();
-    }
-  }
-
-  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-  public static class drainReplicationTable_result implements org.apache.thrift.TBase<drainReplicationTable_result, drainReplicationTable_result._Fields>, java.io.Serializable, Cloneable, Comparable<drainReplicationTable_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("drainReplicationTable_result");
-
-    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.BOOL, (short)0);
-    private static final org.apache.thrift.protocol.TField TNASE_FIELD_DESC = new org.apache.thrift.protocol.TField("tnase", org.apache.thrift.protocol.TType.STRUCT, (short)1);
-
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new drainReplicationTable_resultStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new drainReplicationTable_resultTupleSchemeFactory();
-
-    public boolean success; // required
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException tnase; // 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"),
-      TNASE((short)1, "tnase");
-
-      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: // TNASE
-            return TNASE;
-          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;
-      }
-
-      @Override
-      public short getThriftFieldId() {
-        return _thriftId;
-      }
-
-      @Override
-      public java.lang.String getFieldName() {
-        return _fieldName;
-      }
-    }
-
-    // isset id assignments
-    private static final int __SUCCESS_ISSET_ID = 0;
-    private byte __isset_bitfield = 0;
-    public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
-    static {
-      java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
-      tmpMap.put(_Fields.TNASE, new org.apache.thrift.meta_data.FieldMetaData("tnase", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException.class)));
-      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(drainReplicationTable_result.class, metaDataMap);
-    }
-
-    public drainReplicationTable_result() {
-    }
-
-    public drainReplicationTable_result(
-      boolean success,
-      org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException tnase)
-    {
-      this();
-      this.success = success;
-      setSuccessIsSet(true);
-      this.tnase = tnase;
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public drainReplicationTable_result(drainReplicationTable_result other) {
-      __isset_bitfield = other.__isset_bitfield;
-      this.success = other.success;
-      if (other.isSetTnase()) {
-        this.tnase = new org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException(other.tnase);
-      }
-    }
-
-    @Override
-    public drainReplicationTable_result deepCopy() {
-      return new drainReplicationTable_result(this);
-    }
-
-    @Override
-    public void clear() {
-      setSuccessIsSet(false);
-      this.success = false;
-      this.tnase = null;
-    }
-
-    public boolean isSuccess() {
-      return this.success;
-    }
-
-    public drainReplicationTable_result setSuccess(boolean success) {
-      this.success = success;
-      setSuccessIsSet(true);
-      return this;
-    }
-
-    public void unsetSuccess() {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __SUCCESS_ISSET_ID);
-    }
-
-    /** Returns true if field success is set (has been assigned a value) and false otherwise */
-    public boolean isSetSuccess() {
-      return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __SUCCESS_ISSET_ID);
-    }
-
-    public void setSuccessIsSet(boolean value) {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __SUCCESS_ISSET_ID, value);
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException getTnase() {
-      return this.tnase;
-    }
-
-    public drainReplicationTable_result setTnase(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException tnase) {
-      this.tnase = tnase;
-      return this;
-    }
-
-    public void unsetTnase() {
-      this.tnase = null;
-    }
-
-    /** Returns true if field tnase is set (has been assigned a value) and false otherwise */
-    public boolean isSetTnase() {
-      return this.tnase != null;
-    }
-
-    public void setTnaseIsSet(boolean value) {
-      if (!value) {
-        this.tnase = null;
-      }
-    }
-
-    @Override
-    public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
-      switch (field) {
-      case SUCCESS:
-        if (value == null) {
-          unsetSuccess();
-        } else {
-          setSuccess((java.lang.Boolean)value);
-        }
-        break;
-
-      case TNASE:
-        if (value == null) {
-          unsetTnase();
-        } else {
-          setTnase((org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException)value);
-        }
-        break;
-
-      }
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    @Override
-    public java.lang.Object getFieldValue(_Fields field) {
-      switch (field) {
-      case SUCCESS:
-        return isSuccess();
-
-      case TNASE:
-        return getTnase();
-
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
-    @Override
-    public boolean isSet(_Fields field) {
-      if (field == null) {
-        throw new java.lang.IllegalArgumentException();
-      }
-
-      switch (field) {
-      case SUCCESS:
-        return isSetSuccess();
-      case TNASE:
-        return isSetTnase();
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(java.lang.Object that) {
-      if (that instanceof drainReplicationTable_result)
-        return this.equals((drainReplicationTable_result)that);
-      return false;
-    }
-
-    public boolean equals(drainReplicationTable_result that) {
-      if (that == null)
-        return false;
-      if (this == that)
-        return true;
-
-      boolean this_present_success = true;
-      boolean that_present_success = true;
-      if (this_present_success || that_present_success) {
-        if (!(this_present_success && that_present_success))
-          return false;
-        if (this.success != that.success)
-          return false;
-      }
-
-      boolean this_present_tnase = true && this.isSetTnase();
-      boolean that_present_tnase = true && that.isSetTnase();
-      if (this_present_tnase || that_present_tnase) {
-        if (!(this_present_tnase && that_present_tnase))
-          return false;
-        if (!this.tnase.equals(that.tnase))
-          return false;
-      }
-
-      return true;
-    }
-
-    @Override
-    public int hashCode() {
-      int hashCode = 1;
-
-      hashCode = hashCode * 8191 + ((success) ? 131071 : 524287);
-
-      hashCode = hashCode * 8191 + ((isSetTnase()) ? 131071 : 524287);
-      if (isSetTnase())
-        hashCode = hashCode * 8191 + tnase.hashCode();
-
-      return hashCode;
-    }
-
-    @Override
-    public int compareTo(drainReplicationTable_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(isSetTnase(), other.isSetTnase());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetTnase()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tnase, other.tnase);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      return 0;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    @Override
-    public _Fields fieldForId(int fieldId) {
-      return _Fields.findByThriftId(fieldId);
-    }
-
-    @Override
-    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("drainReplicationTable_result(");
-      boolean first = true;
-
-      sb.append("success:");
-      sb.append(this.success);
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("tnase:");
-      if (this.tnase == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.tnase);
-      }
-      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 {
-        // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
-        __isset_bitfield = 0;
-        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
-      } catch (org.apache.thrift.TException te) {
-        throw new java.io.IOException(te);
-      }
-    }
-
-    private static class drainReplicationTable_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public drainReplicationTable_resultStandardScheme getScheme() {
-        return new drainReplicationTable_resultStandardScheme();
-      }
-    }
-
-    private static class drainReplicationTable_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<drainReplicationTable_result> {
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol iprot, drainReplicationTable_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.BOOL) {
-                struct.success = iprot.readBool();
-                struct.setSuccessIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 1: // TNASE
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.tnase = new org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException();
-                struct.tnase.read(iprot);
-                struct.setTnaseIsSet(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();
-      }
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol oprot, drainReplicationTable_result struct) throws org.apache.thrift.TException {
-        struct.validate();
-
-        oprot.writeStructBegin(STRUCT_DESC);
-        if (struct.isSetSuccess()) {
-          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
-          oprot.writeBool(struct.success);
-          oprot.writeFieldEnd();
-        }
-        if (struct.tnase != null) {
-          oprot.writeFieldBegin(TNASE_FIELD_DESC);
-          struct.tnase.write(oprot);
-          oprot.writeFieldEnd();
-        }
-        oprot.writeFieldStop();
-        oprot.writeStructEnd();
-      }
-
-    }
-
-    private static class drainReplicationTable_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public drainReplicationTable_resultTupleScheme getScheme() {
-        return new drainReplicationTable_resultTupleScheme();
-      }
-    }
-
-    private static class drainReplicationTable_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<drainReplicationTable_result> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, drainReplicationTable_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.isSetTnase()) {
-          optionals.set(1);
-        }
-        oprot.writeBitSet(optionals, 2);
-        if (struct.isSetSuccess()) {
-          oprot.writeBool(struct.success);
-        }
-        if (struct.isSetTnase()) {
-          struct.tnase.write(oprot);
-        }
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, drainReplicationTable_result struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
-        java.util.BitSet incoming = iprot.readBitSet(2);
-        if (incoming.get(0)) {
-          struct.success = iprot.readBool();
-          struct.setSuccessIsSet(true);
-        }
-        if (incoming.get(1)) {
-          struct.tnase = new org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException();
-          struct.tnase.read(iprot);
-          struct.setTnaseIsSet(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();
-    }
-  }
-
   private static void unusedMethod() {}
 }
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/ManagerMonitorInfo.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/ManagerMonitorInfo.java
index 8d2c3ce..bd7eda3 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/ManagerMonitorInfo.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/ManagerMonitorInfo.java
@@ -41,8 +41,8 @@
   private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new ManagerMonitorInfoStandardSchemeFactory();
   private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new ManagerMonitorInfoTupleSchemeFactory();
 
-  public @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,org.apache.accumulo.core.master.thrift.TableInfo> tableMap; // required
-  public @org.apache.thrift.annotation.Nullable java.util.List<org.apache.accumulo.core.master.thrift.TabletServerStatus> tServerInfo; // required
+  public @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,TableInfo> tableMap; // required
+  public @org.apache.thrift.annotation.Nullable java.util.List<TabletServerStatus> tServerInfo; // required
   public @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,java.lang.Byte> badTServers; // required
   /**
    * 
@@ -57,7 +57,7 @@
   public int unassignedTablets; // required
   public @org.apache.thrift.annotation.Nullable java.util.Set<java.lang.String> serversShuttingDown; // required
   public @org.apache.thrift.annotation.Nullable java.util.List<DeadServer> deadTabletServers; // required
-  public @org.apache.thrift.annotation.Nullable java.util.List<org.apache.accumulo.core.master.thrift.BulkImportStatus> bulkImports; // required
+  public @org.apache.thrift.annotation.Nullable java.util.List<BulkImportStatus> bulkImports; // 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 {
@@ -162,10 +162,10 @@
     tmpMap.put(_Fields.TABLE_MAP, new org.apache.thrift.meta_data.FieldMetaData("tableMap", 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.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.master.thrift.TableInfo.class))));
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TableInfo.class))));
     tmpMap.put(_Fields.T_SERVER_INFO, new org.apache.thrift.meta_data.FieldMetaData("tServerInfo", org.apache.thrift.TFieldRequirementType.DEFAULT, 
         new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
-            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.master.thrift.TabletServerStatus.class))));
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TabletServerStatus.class))));
     tmpMap.put(_Fields.BAD_TSERVERS, new org.apache.thrift.meta_data.FieldMetaData("badTServers", 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), 
@@ -184,7 +184,7 @@
             new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, DeadServer.class))));
     tmpMap.put(_Fields.BULK_IMPORTS, new org.apache.thrift.meta_data.FieldMetaData("bulkImports", org.apache.thrift.TFieldRequirementType.DEFAULT, 
         new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
-            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.master.thrift.BulkImportStatus.class))));
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, BulkImportStatus.class))));
     metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ManagerMonitorInfo.class, metaDataMap);
   }
@@ -193,15 +193,15 @@
   }
 
   public ManagerMonitorInfo(
-    java.util.Map<java.lang.String,org.apache.accumulo.core.master.thrift.TableInfo> tableMap,
-    java.util.List<org.apache.accumulo.core.master.thrift.TabletServerStatus> tServerInfo,
+    java.util.Map<java.lang.String,TableInfo> tableMap,
+    java.util.List<TabletServerStatus> tServerInfo,
     java.util.Map<java.lang.String,java.lang.Byte> badTServers,
     ManagerState state,
     ManagerGoalState goalState,
     int unassignedTablets,
     java.util.Set<java.lang.String> serversShuttingDown,
     java.util.List<DeadServer> deadTabletServers,
-    java.util.List<org.apache.accumulo.core.master.thrift.BulkImportStatus> bulkImports)
+    java.util.List<BulkImportStatus> bulkImports)
   {
     this();
     this.tableMap = tableMap;
@@ -222,24 +222,24 @@
   public ManagerMonitorInfo(ManagerMonitorInfo other) {
     __isset_bitfield = other.__isset_bitfield;
     if (other.isSetTableMap()) {
-      java.util.Map<java.lang.String,org.apache.accumulo.core.master.thrift.TableInfo> __this__tableMap = new java.util.HashMap<java.lang.String,org.apache.accumulo.core.master.thrift.TableInfo>(other.tableMap.size());
-      for (java.util.Map.Entry<java.lang.String, org.apache.accumulo.core.master.thrift.TableInfo> other_element : other.tableMap.entrySet()) {
+      java.util.Map<java.lang.String,TableInfo> __this__tableMap = new java.util.HashMap<java.lang.String,TableInfo>(other.tableMap.size());
+      for (java.util.Map.Entry<java.lang.String, TableInfo> other_element : other.tableMap.entrySet()) {
 
         java.lang.String other_element_key = other_element.getKey();
-        org.apache.accumulo.core.master.thrift.TableInfo other_element_value = other_element.getValue();
+        TableInfo other_element_value = other_element.getValue();
 
         java.lang.String __this__tableMap_copy_key = other_element_key;
 
-        org.apache.accumulo.core.master.thrift.TableInfo __this__tableMap_copy_value = new org.apache.accumulo.core.master.thrift.TableInfo(other_element_value);
+        TableInfo __this__tableMap_copy_value = new TableInfo(other_element_value);
 
         __this__tableMap.put(__this__tableMap_copy_key, __this__tableMap_copy_value);
       }
       this.tableMap = __this__tableMap;
     }
     if (other.isSetTServerInfo()) {
-      java.util.List<org.apache.accumulo.core.master.thrift.TabletServerStatus> __this__tServerInfo = new java.util.ArrayList<org.apache.accumulo.core.master.thrift.TabletServerStatus>(other.tServerInfo.size());
-      for (org.apache.accumulo.core.master.thrift.TabletServerStatus other_element : other.tServerInfo) {
-        __this__tServerInfo.add(new org.apache.accumulo.core.master.thrift.TabletServerStatus(other_element));
+      java.util.List<TabletServerStatus> __this__tServerInfo = new java.util.ArrayList<TabletServerStatus>(other.tServerInfo.size());
+      for (TabletServerStatus other_element : other.tServerInfo) {
+        __this__tServerInfo.add(new TabletServerStatus(other_element));
       }
       this.tServerInfo = __this__tServerInfo;
     }
@@ -266,9 +266,9 @@
       this.deadTabletServers = __this__deadTabletServers;
     }
     if (other.isSetBulkImports()) {
-      java.util.List<org.apache.accumulo.core.master.thrift.BulkImportStatus> __this__bulkImports = new java.util.ArrayList<org.apache.accumulo.core.master.thrift.BulkImportStatus>(other.bulkImports.size());
-      for (org.apache.accumulo.core.master.thrift.BulkImportStatus other_element : other.bulkImports) {
-        __this__bulkImports.add(new org.apache.accumulo.core.master.thrift.BulkImportStatus(other_element));
+      java.util.List<BulkImportStatus> __this__bulkImports = new java.util.ArrayList<BulkImportStatus>(other.bulkImports.size());
+      for (BulkImportStatus other_element : other.bulkImports) {
+        __this__bulkImports.add(new BulkImportStatus(other_element));
       }
       this.bulkImports = __this__bulkImports;
     }
@@ -297,19 +297,19 @@
     return (this.tableMap == null) ? 0 : this.tableMap.size();
   }
 
-  public void putToTableMap(java.lang.String key, org.apache.accumulo.core.master.thrift.TableInfo val) {
+  public void putToTableMap(java.lang.String key, TableInfo val) {
     if (this.tableMap == null) {
-      this.tableMap = new java.util.HashMap<java.lang.String,org.apache.accumulo.core.master.thrift.TableInfo>();
+      this.tableMap = new java.util.HashMap<java.lang.String,TableInfo>();
     }
     this.tableMap.put(key, val);
   }
 
   @org.apache.thrift.annotation.Nullable
-  public java.util.Map<java.lang.String,org.apache.accumulo.core.master.thrift.TableInfo> getTableMap() {
+  public java.util.Map<java.lang.String,TableInfo> getTableMap() {
     return this.tableMap;
   }
 
-  public ManagerMonitorInfo setTableMap(@org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,org.apache.accumulo.core.master.thrift.TableInfo> tableMap) {
+  public ManagerMonitorInfo setTableMap(@org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,TableInfo> tableMap) {
     this.tableMap = tableMap;
     return this;
   }
@@ -334,23 +334,23 @@
   }
 
   @org.apache.thrift.annotation.Nullable
-  public java.util.Iterator<org.apache.accumulo.core.master.thrift.TabletServerStatus> getTServerInfoIterator() {
+  public java.util.Iterator<TabletServerStatus> getTServerInfoIterator() {
     return (this.tServerInfo == null) ? null : this.tServerInfo.iterator();
   }
 
-  public void addToTServerInfo(org.apache.accumulo.core.master.thrift.TabletServerStatus elem) {
+  public void addToTServerInfo(TabletServerStatus elem) {
     if (this.tServerInfo == null) {
-      this.tServerInfo = new java.util.ArrayList<org.apache.accumulo.core.master.thrift.TabletServerStatus>();
+      this.tServerInfo = new java.util.ArrayList<TabletServerStatus>();
     }
     this.tServerInfo.add(elem);
   }
 
   @org.apache.thrift.annotation.Nullable
-  public java.util.List<org.apache.accumulo.core.master.thrift.TabletServerStatus> getTServerInfo() {
+  public java.util.List<TabletServerStatus> getTServerInfo() {
     return this.tServerInfo;
   }
 
-  public ManagerMonitorInfo setTServerInfo(@org.apache.thrift.annotation.Nullable java.util.List<org.apache.accumulo.core.master.thrift.TabletServerStatus> tServerInfo) {
+  public ManagerMonitorInfo setTServerInfo(@org.apache.thrift.annotation.Nullable java.util.List<TabletServerStatus> tServerInfo) {
     this.tServerInfo = tServerInfo;
     return this;
   }
@@ -582,23 +582,23 @@
   }
 
   @org.apache.thrift.annotation.Nullable
-  public java.util.Iterator<org.apache.accumulo.core.master.thrift.BulkImportStatus> getBulkImportsIterator() {
+  public java.util.Iterator<BulkImportStatus> getBulkImportsIterator() {
     return (this.bulkImports == null) ? null : this.bulkImports.iterator();
   }
 
-  public void addToBulkImports(org.apache.accumulo.core.master.thrift.BulkImportStatus elem) {
+  public void addToBulkImports(BulkImportStatus elem) {
     if (this.bulkImports == null) {
-      this.bulkImports = new java.util.ArrayList<org.apache.accumulo.core.master.thrift.BulkImportStatus>();
+      this.bulkImports = new java.util.ArrayList<BulkImportStatus>();
     }
     this.bulkImports.add(elem);
   }
 
   @org.apache.thrift.annotation.Nullable
-  public java.util.List<org.apache.accumulo.core.master.thrift.BulkImportStatus> getBulkImports() {
+  public java.util.List<BulkImportStatus> getBulkImports() {
     return this.bulkImports;
   }
 
-  public ManagerMonitorInfo setBulkImports(@org.apache.thrift.annotation.Nullable java.util.List<org.apache.accumulo.core.master.thrift.BulkImportStatus> bulkImports) {
+  public ManagerMonitorInfo setBulkImports(@org.apache.thrift.annotation.Nullable java.util.List<BulkImportStatus> bulkImports) {
     this.bulkImports = bulkImports;
     return this;
   }
@@ -625,7 +625,7 @@
       if (value == null) {
         unsetTableMap();
       } else {
-        setTableMap((java.util.Map<java.lang.String,org.apache.accumulo.core.master.thrift.TableInfo>)value);
+        setTableMap((java.util.Map<java.lang.String,TableInfo>)value);
       }
       break;
 
@@ -633,7 +633,7 @@
       if (value == null) {
         unsetTServerInfo();
       } else {
-        setTServerInfo((java.util.List<org.apache.accumulo.core.master.thrift.TabletServerStatus>)value);
+        setTServerInfo((java.util.List<TabletServerStatus>)value);
       }
       break;
 
@@ -689,7 +689,7 @@
       if (value == null) {
         unsetBulkImports();
       } else {
-        setBulkImports((java.util.List<org.apache.accumulo.core.master.thrift.BulkImportStatus>)value);
+        setBulkImports((java.util.List<BulkImportStatus>)value);
       }
       break;
 
@@ -1138,16 +1138,16 @@
           case 1: // TABLE_MAP
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map8 = iprot.readMapBegin();
-                struct.tableMap = new java.util.HashMap<java.lang.String,org.apache.accumulo.core.master.thrift.TableInfo>(2*_map8.size);
-                @org.apache.thrift.annotation.Nullable java.lang.String _key9;
-                @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.master.thrift.TableInfo _val10;
-                for (int _i11 = 0; _i11 < _map8.size; ++_i11)
+                org.apache.thrift.protocol.TMap _map34 = iprot.readMapBegin();
+                struct.tableMap = new java.util.HashMap<java.lang.String,TableInfo>(2*_map34.size);
+                @org.apache.thrift.annotation.Nullable java.lang.String _key35;
+                @org.apache.thrift.annotation.Nullable TableInfo _val36;
+                for (int _i37 = 0; _i37 < _map34.size; ++_i37)
                 {
-                  _key9 = iprot.readString();
-                  _val10 = new org.apache.accumulo.core.master.thrift.TableInfo();
-                  _val10.read(iprot);
-                  struct.tableMap.put(_key9, _val10);
+                  _key35 = iprot.readString();
+                  _val36 = new TableInfo();
+                  _val36.read(iprot);
+                  struct.tableMap.put(_key35, _val36);
                 }
                 iprot.readMapEnd();
               }
@@ -1159,14 +1159,14 @@
           case 2: // T_SERVER_INFO
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list12 = iprot.readListBegin();
-                struct.tServerInfo = new java.util.ArrayList<org.apache.accumulo.core.master.thrift.TabletServerStatus>(_list12.size);
-                @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.master.thrift.TabletServerStatus _elem13;
-                for (int _i14 = 0; _i14 < _list12.size; ++_i14)
+                org.apache.thrift.protocol.TList _list38 = iprot.readListBegin();
+                struct.tServerInfo = new java.util.ArrayList<TabletServerStatus>(_list38.size);
+                @org.apache.thrift.annotation.Nullable TabletServerStatus _elem39;
+                for (int _i40 = 0; _i40 < _list38.size; ++_i40)
                 {
-                  _elem13 = new org.apache.accumulo.core.master.thrift.TabletServerStatus();
-                  _elem13.read(iprot);
-                  struct.tServerInfo.add(_elem13);
+                  _elem39 = new TabletServerStatus();
+                  _elem39.read(iprot);
+                  struct.tServerInfo.add(_elem39);
                 }
                 iprot.readListEnd();
               }
@@ -1178,15 +1178,15 @@
           case 3: // BAD_TSERVERS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map15 = iprot.readMapBegin();
-                struct.badTServers = new java.util.HashMap<java.lang.String,java.lang.Byte>(2*_map15.size);
-                @org.apache.thrift.annotation.Nullable java.lang.String _key16;
-                byte _val17;
-                for (int _i18 = 0; _i18 < _map15.size; ++_i18)
+                org.apache.thrift.protocol.TMap _map41 = iprot.readMapBegin();
+                struct.badTServers = new java.util.HashMap<java.lang.String,java.lang.Byte>(2*_map41.size);
+                @org.apache.thrift.annotation.Nullable java.lang.String _key42;
+                byte _val43;
+                for (int _i44 = 0; _i44 < _map41.size; ++_i44)
                 {
-                  _key16 = iprot.readString();
-                  _val17 = iprot.readByte();
-                  struct.badTServers.put(_key16, _val17);
+                  _key42 = iprot.readString();
+                  _val43 = iprot.readByte();
+                  struct.badTServers.put(_key42, _val43);
                 }
                 iprot.readMapEnd();
               }
@@ -1222,13 +1222,13 @@
           case 7: // SERVERS_SHUTTING_DOWN
             if (schemeField.type == org.apache.thrift.protocol.TType.SET) {
               {
-                org.apache.thrift.protocol.TSet _set19 = iprot.readSetBegin();
-                struct.serversShuttingDown = new java.util.HashSet<java.lang.String>(2*_set19.size);
-                @org.apache.thrift.annotation.Nullable java.lang.String _elem20;
-                for (int _i21 = 0; _i21 < _set19.size; ++_i21)
+                org.apache.thrift.protocol.TSet _set45 = iprot.readSetBegin();
+                struct.serversShuttingDown = 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)
                 {
-                  _elem20 = iprot.readString();
-                  struct.serversShuttingDown.add(_elem20);
+                  _elem46 = iprot.readString();
+                  struct.serversShuttingDown.add(_elem46);
                 }
                 iprot.readSetEnd();
               }
@@ -1240,14 +1240,14 @@
           case 8: // DEAD_TABLET_SERVERS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list22 = iprot.readListBegin();
-                struct.deadTabletServers = new java.util.ArrayList<DeadServer>(_list22.size);
-                @org.apache.thrift.annotation.Nullable DeadServer _elem23;
-                for (int _i24 = 0; _i24 < _list22.size; ++_i24)
+                org.apache.thrift.protocol.TList _list48 = iprot.readListBegin();
+                struct.deadTabletServers = new java.util.ArrayList<DeadServer>(_list48.size);
+                @org.apache.thrift.annotation.Nullable DeadServer _elem49;
+                for (int _i50 = 0; _i50 < _list48.size; ++_i50)
                 {
-                  _elem23 = new DeadServer();
-                  _elem23.read(iprot);
-                  struct.deadTabletServers.add(_elem23);
+                  _elem49 = new DeadServer();
+                  _elem49.read(iprot);
+                  struct.deadTabletServers.add(_elem49);
                 }
                 iprot.readListEnd();
               }
@@ -1259,14 +1259,14 @@
           case 9: // BULK_IMPORTS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list25 = iprot.readListBegin();
-                struct.bulkImports = new java.util.ArrayList<org.apache.accumulo.core.master.thrift.BulkImportStatus>(_list25.size);
-                @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.master.thrift.BulkImportStatus _elem26;
-                for (int _i27 = 0; _i27 < _list25.size; ++_i27)
+                org.apache.thrift.protocol.TList _list51 = iprot.readListBegin();
+                struct.bulkImports = new java.util.ArrayList<BulkImportStatus>(_list51.size);
+                @org.apache.thrift.annotation.Nullable BulkImportStatus _elem52;
+                for (int _i53 = 0; _i53 < _list51.size; ++_i53)
                 {
-                  _elem26 = new org.apache.accumulo.core.master.thrift.BulkImportStatus();
-                  _elem26.read(iprot);
-                  struct.bulkImports.add(_elem26);
+                  _elem52 = new BulkImportStatus();
+                  _elem52.read(iprot);
+                  struct.bulkImports.add(_elem52);
                 }
                 iprot.readListEnd();
               }
@@ -1295,10 +1295,10 @@
         oprot.writeFieldBegin(TABLE_MAP_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.tableMap.size()));
-          for (java.util.Map.Entry<java.lang.String, org.apache.accumulo.core.master.thrift.TableInfo> _iter28 : struct.tableMap.entrySet())
+          for (java.util.Map.Entry<java.lang.String, TableInfo> _iter54 : struct.tableMap.entrySet())
           {
-            oprot.writeString(_iter28.getKey());
-            _iter28.getValue().write(oprot);
+            oprot.writeString(_iter54.getKey());
+            _iter54.getValue().write(oprot);
           }
           oprot.writeMapEnd();
         }
@@ -1308,9 +1308,9 @@
         oprot.writeFieldBegin(T_SERVER_INFO_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.tServerInfo.size()));
-          for (org.apache.accumulo.core.master.thrift.TabletServerStatus _iter29 : struct.tServerInfo)
+          for (TabletServerStatus _iter55 : struct.tServerInfo)
           {
-            _iter29.write(oprot);
+            _iter55.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -1320,10 +1320,10 @@
         oprot.writeFieldBegin(BAD_TSERVERS_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.BYTE, struct.badTServers.size()));
-          for (java.util.Map.Entry<java.lang.String, java.lang.Byte> _iter30 : struct.badTServers.entrySet())
+          for (java.util.Map.Entry<java.lang.String, java.lang.Byte> _iter56 : struct.badTServers.entrySet())
           {
-            oprot.writeString(_iter30.getKey());
-            oprot.writeByte(_iter30.getValue());
+            oprot.writeString(_iter56.getKey());
+            oprot.writeByte(_iter56.getValue());
           }
           oprot.writeMapEnd();
         }
@@ -1346,9 +1346,9 @@
         oprot.writeFieldBegin(SERVERS_SHUTTING_DOWN_FIELD_DESC);
         {
           oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRING, struct.serversShuttingDown.size()));
-          for (java.lang.String _iter31 : struct.serversShuttingDown)
+          for (java.lang.String _iter57 : struct.serversShuttingDown)
           {
-            oprot.writeString(_iter31);
+            oprot.writeString(_iter57);
           }
           oprot.writeSetEnd();
         }
@@ -1358,9 +1358,9 @@
         oprot.writeFieldBegin(DEAD_TABLET_SERVERS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.deadTabletServers.size()));
-          for (DeadServer _iter32 : struct.deadTabletServers)
+          for (DeadServer _iter58 : struct.deadTabletServers)
           {
-            _iter32.write(oprot);
+            _iter58.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -1370,9 +1370,9 @@
         oprot.writeFieldBegin(BULK_IMPORTS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.bulkImports.size()));
-          for (org.apache.accumulo.core.master.thrift.BulkImportStatus _iter33 : struct.bulkImports)
+          for (BulkImportStatus _iter59 : struct.bulkImports)
           {
-            _iter33.write(oprot);
+            _iter59.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -1428,29 +1428,29 @@
       if (struct.isSetTableMap()) {
         {
           oprot.writeI32(struct.tableMap.size());
-          for (java.util.Map.Entry<java.lang.String, org.apache.accumulo.core.master.thrift.TableInfo> _iter34 : struct.tableMap.entrySet())
+          for (java.util.Map.Entry<java.lang.String, TableInfo> _iter60 : struct.tableMap.entrySet())
           {
-            oprot.writeString(_iter34.getKey());
-            _iter34.getValue().write(oprot);
+            oprot.writeString(_iter60.getKey());
+            _iter60.getValue().write(oprot);
           }
         }
       }
       if (struct.isSetTServerInfo()) {
         {
           oprot.writeI32(struct.tServerInfo.size());
-          for (org.apache.accumulo.core.master.thrift.TabletServerStatus _iter35 : struct.tServerInfo)
+          for (TabletServerStatus _iter61 : struct.tServerInfo)
           {
-            _iter35.write(oprot);
+            _iter61.write(oprot);
           }
         }
       }
       if (struct.isSetBadTServers()) {
         {
           oprot.writeI32(struct.badTServers.size());
-          for (java.util.Map.Entry<java.lang.String, java.lang.Byte> _iter36 : struct.badTServers.entrySet())
+          for (java.util.Map.Entry<java.lang.String, java.lang.Byte> _iter62 : struct.badTServers.entrySet())
           {
-            oprot.writeString(_iter36.getKey());
-            oprot.writeByte(_iter36.getValue());
+            oprot.writeString(_iter62.getKey());
+            oprot.writeByte(_iter62.getValue());
           }
         }
       }
@@ -1466,27 +1466,27 @@
       if (struct.isSetServersShuttingDown()) {
         {
           oprot.writeI32(struct.serversShuttingDown.size());
-          for (java.lang.String _iter37 : struct.serversShuttingDown)
+          for (java.lang.String _iter63 : struct.serversShuttingDown)
           {
-            oprot.writeString(_iter37);
+            oprot.writeString(_iter63);
           }
         }
       }
       if (struct.isSetDeadTabletServers()) {
         {
           oprot.writeI32(struct.deadTabletServers.size());
-          for (DeadServer _iter38 : struct.deadTabletServers)
+          for (DeadServer _iter64 : struct.deadTabletServers)
           {
-            _iter38.write(oprot);
+            _iter64.write(oprot);
           }
         }
       }
       if (struct.isSetBulkImports()) {
         {
           oprot.writeI32(struct.bulkImports.size());
-          for (org.apache.accumulo.core.master.thrift.BulkImportStatus _iter39 : struct.bulkImports)
+          for (BulkImportStatus _iter65 : struct.bulkImports)
           {
-            _iter39.write(oprot);
+            _iter65.write(oprot);
           }
         }
       }
@@ -1498,45 +1498,45 @@
       java.util.BitSet incoming = iprot.readBitSet(9);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TMap _map40 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT); 
-          struct.tableMap = new java.util.HashMap<java.lang.String,org.apache.accumulo.core.master.thrift.TableInfo>(2*_map40.size);
-          @org.apache.thrift.annotation.Nullable java.lang.String _key41;
-          @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.master.thrift.TableInfo _val42;
-          for (int _i43 = 0; _i43 < _map40.size; ++_i43)
+          org.apache.thrift.protocol.TMap _map66 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT); 
+          struct.tableMap = new java.util.HashMap<java.lang.String,TableInfo>(2*_map66.size);
+          @org.apache.thrift.annotation.Nullable java.lang.String _key67;
+          @org.apache.thrift.annotation.Nullable TableInfo _val68;
+          for (int _i69 = 0; _i69 < _map66.size; ++_i69)
           {
-            _key41 = iprot.readString();
-            _val42 = new org.apache.accumulo.core.master.thrift.TableInfo();
-            _val42.read(iprot);
-            struct.tableMap.put(_key41, _val42);
+            _key67 = iprot.readString();
+            _val68 = new TableInfo();
+            _val68.read(iprot);
+            struct.tableMap.put(_key67, _val68);
           }
         }
         struct.setTableMapIsSet(true);
       }
       if (incoming.get(1)) {
         {
-          org.apache.thrift.protocol.TList _list44 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
-          struct.tServerInfo = new java.util.ArrayList<org.apache.accumulo.core.master.thrift.TabletServerStatus>(_list44.size);
-          @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.master.thrift.TabletServerStatus _elem45;
-          for (int _i46 = 0; _i46 < _list44.size; ++_i46)
+          org.apache.thrift.protocol.TList _list70 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+          struct.tServerInfo = new java.util.ArrayList<TabletServerStatus>(_list70.size);
+          @org.apache.thrift.annotation.Nullable TabletServerStatus _elem71;
+          for (int _i72 = 0; _i72 < _list70.size; ++_i72)
           {
-            _elem45 = new org.apache.accumulo.core.master.thrift.TabletServerStatus();
-            _elem45.read(iprot);
-            struct.tServerInfo.add(_elem45);
+            _elem71 = new TabletServerStatus();
+            _elem71.read(iprot);
+            struct.tServerInfo.add(_elem71);
           }
         }
         struct.setTServerInfoIsSet(true);
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TMap _map47 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.BYTE); 
-          struct.badTServers = new java.util.HashMap<java.lang.String,java.lang.Byte>(2*_map47.size);
-          @org.apache.thrift.annotation.Nullable java.lang.String _key48;
-          byte _val49;
-          for (int _i50 = 0; _i50 < _map47.size; ++_i50)
+          org.apache.thrift.protocol.TMap _map73 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.BYTE); 
+          struct.badTServers = new java.util.HashMap<java.lang.String,java.lang.Byte>(2*_map73.size);
+          @org.apache.thrift.annotation.Nullable java.lang.String _key74;
+          byte _val75;
+          for (int _i76 = 0; _i76 < _map73.size; ++_i76)
           {
-            _key48 = iprot.readString();
-            _val49 = iprot.readByte();
-            struct.badTServers.put(_key48, _val49);
+            _key74 = iprot.readString();
+            _val75 = iprot.readByte();
+            struct.badTServers.put(_key74, _val75);
           }
         }
         struct.setBadTServersIsSet(true);
@@ -1555,41 +1555,41 @@
       }
       if (incoming.get(6)) {
         {
-          org.apache.thrift.protocol.TSet _set51 = iprot.readSetBegin(org.apache.thrift.protocol.TType.STRING);
-          struct.serversShuttingDown = new java.util.HashSet<java.lang.String>(2*_set51.size);
-          @org.apache.thrift.annotation.Nullable java.lang.String _elem52;
-          for (int _i53 = 0; _i53 < _set51.size; ++_i53)
+          org.apache.thrift.protocol.TSet _set77 = iprot.readSetBegin(org.apache.thrift.protocol.TType.STRING);
+          struct.serversShuttingDown = new java.util.HashSet<java.lang.String>(2*_set77.size);
+          @org.apache.thrift.annotation.Nullable java.lang.String _elem78;
+          for (int _i79 = 0; _i79 < _set77.size; ++_i79)
           {
-            _elem52 = iprot.readString();
-            struct.serversShuttingDown.add(_elem52);
+            _elem78 = iprot.readString();
+            struct.serversShuttingDown.add(_elem78);
           }
         }
         struct.setServersShuttingDownIsSet(true);
       }
       if (incoming.get(7)) {
         {
-          org.apache.thrift.protocol.TList _list54 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
-          struct.deadTabletServers = new java.util.ArrayList<DeadServer>(_list54.size);
-          @org.apache.thrift.annotation.Nullable DeadServer _elem55;
-          for (int _i56 = 0; _i56 < _list54.size; ++_i56)
+          org.apache.thrift.protocol.TList _list80 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+          struct.deadTabletServers = new java.util.ArrayList<DeadServer>(_list80.size);
+          @org.apache.thrift.annotation.Nullable DeadServer _elem81;
+          for (int _i82 = 0; _i82 < _list80.size; ++_i82)
           {
-            _elem55 = new DeadServer();
-            _elem55.read(iprot);
-            struct.deadTabletServers.add(_elem55);
+            _elem81 = new DeadServer();
+            _elem81.read(iprot);
+            struct.deadTabletServers.add(_elem81);
           }
         }
         struct.setDeadTabletServersIsSet(true);
       }
       if (incoming.get(8)) {
         {
-          org.apache.thrift.protocol.TList _list57 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
-          struct.bulkImports = new java.util.ArrayList<org.apache.accumulo.core.master.thrift.BulkImportStatus>(_list57.size);
-          @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.master.thrift.BulkImportStatus _elem58;
-          for (int _i59 = 0; _i59 < _list57.size; ++_i59)
+          org.apache.thrift.protocol.TList _list83 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+          struct.bulkImports = new java.util.ArrayList<BulkImportStatus>(_list83.size);
+          @org.apache.thrift.annotation.Nullable BulkImportStatus _elem84;
+          for (int _i85 = 0; _i85 < _list83.size; ++_i85)
           {
-            _elem58 = new org.apache.accumulo.core.master.thrift.BulkImportStatus();
-            _elem58.read(iprot);
-            struct.bulkImports.add(_elem58);
+            _elem84 = new BulkImportStatus();
+            _elem84.read(iprot);
+            struct.bulkImports.add(_elem84);
           }
         }
         struct.setBulkImportsIsSet(true);
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/RecoveryException.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/RecoveryException.java
deleted file mode 100644
index 32eb228..0000000
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/RecoveryException.java
+++ /dev/null
@@ -1,406 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/**
- * Autogenerated by Thrift Compiler (0.17.0)
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- *  @generated
- */
-package org.apache.accumulo.core.manager.thrift;
-
-@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-public class RecoveryException extends org.apache.thrift.TException implements org.apache.thrift.TBase<RecoveryException, RecoveryException._Fields>, java.io.Serializable, Cloneable, Comparable<RecoveryException> {
-  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("RecoveryException");
-
-  private static final org.apache.thrift.protocol.TField WHY_FIELD_DESC = new org.apache.thrift.protocol.TField("why", org.apache.thrift.protocol.TType.STRING, (short)1);
-
-  private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new RecoveryExceptionStandardSchemeFactory();
-  private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new RecoveryExceptionTupleSchemeFactory();
-
-  public @org.apache.thrift.annotation.Nullable java.lang.String why; // 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 {
-    WHY((short)1, "why");
-
-    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: // WHY
-          return WHY;
-        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;
-    }
-
-    @Override
-    public short getThriftFieldId() {
-      return _thriftId;
-    }
-
-    @Override
-    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.WHY, new org.apache.thrift.meta_data.FieldMetaData("why", 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(RecoveryException.class, metaDataMap);
-  }
-
-  public RecoveryException() {
-  }
-
-  public RecoveryException(
-    java.lang.String why)
-  {
-    this();
-    this.why = why;
-  }
-
-  /**
-   * Performs a deep copy on <i>other</i>.
-   */
-  public RecoveryException(RecoveryException other) {
-    if (other.isSetWhy()) {
-      this.why = other.why;
-    }
-  }
-
-  @Override
-  public RecoveryException deepCopy() {
-    return new RecoveryException(this);
-  }
-
-  @Override
-  public void clear() {
-    this.why = null;
-  }
-
-  @org.apache.thrift.annotation.Nullable
-  public java.lang.String getWhy() {
-    return this.why;
-  }
-
-  public RecoveryException setWhy(@org.apache.thrift.annotation.Nullable java.lang.String why) {
-    this.why = why;
-    return this;
-  }
-
-  public void unsetWhy() {
-    this.why = null;
-  }
-
-  /** Returns true if field why is set (has been assigned a value) and false otherwise */
-  public boolean isSetWhy() {
-    return this.why != null;
-  }
-
-  public void setWhyIsSet(boolean value) {
-    if (!value) {
-      this.why = null;
-    }
-  }
-
-  @Override
-  public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
-    switch (field) {
-    case WHY:
-      if (value == null) {
-        unsetWhy();
-      } else {
-        setWhy((java.lang.String)value);
-      }
-      break;
-
-    }
-  }
-
-  @org.apache.thrift.annotation.Nullable
-  @Override
-  public java.lang.Object getFieldValue(_Fields field) {
-    switch (field) {
-    case WHY:
-      return getWhy();
-
-    }
-    throw new java.lang.IllegalStateException();
-  }
-
-  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
-  @Override
-  public boolean isSet(_Fields field) {
-    if (field == null) {
-      throw new java.lang.IllegalArgumentException();
-    }
-
-    switch (field) {
-    case WHY:
-      return isSetWhy();
-    }
-    throw new java.lang.IllegalStateException();
-  }
-
-  @Override
-  public boolean equals(java.lang.Object that) {
-    if (that instanceof RecoveryException)
-      return this.equals((RecoveryException)that);
-    return false;
-  }
-
-  public boolean equals(RecoveryException that) {
-    if (that == null)
-      return false;
-    if (this == that)
-      return true;
-
-    boolean this_present_why = true && this.isSetWhy();
-    boolean that_present_why = true && that.isSetWhy();
-    if (this_present_why || that_present_why) {
-      if (!(this_present_why && that_present_why))
-        return false;
-      if (!this.why.equals(that.why))
-        return false;
-    }
-
-    return true;
-  }
-
-  @Override
-  public int hashCode() {
-    int hashCode = 1;
-
-    hashCode = hashCode * 8191 + ((isSetWhy()) ? 131071 : 524287);
-    if (isSetWhy())
-      hashCode = hashCode * 8191 + why.hashCode();
-
-    return hashCode;
-  }
-
-  @Override
-  public int compareTo(RecoveryException other) {
-    if (!getClass().equals(other.getClass())) {
-      return getClass().getName().compareTo(other.getClass().getName());
-    }
-
-    int lastComparison = 0;
-
-    lastComparison = java.lang.Boolean.compare(isSetWhy(), other.isSetWhy());
-    if (lastComparison != 0) {
-      return lastComparison;
-    }
-    if (isSetWhy()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.why, other.why);
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-    }
-    return 0;
-  }
-
-  @org.apache.thrift.annotation.Nullable
-  @Override
-  public _Fields fieldForId(int fieldId) {
-    return _Fields.findByThriftId(fieldId);
-  }
-
-  @Override
-  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
-    scheme(iprot).read(iprot, this);
-  }
-
-  @Override
-  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("RecoveryException(");
-    boolean first = true;
-
-    sb.append("why:");
-    if (this.why == null) {
-      sb.append("null");
-    } else {
-      sb.append(this.why);
-    }
-    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 RecoveryExceptionStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-    @Override
-    public RecoveryExceptionStandardScheme getScheme() {
-      return new RecoveryExceptionStandardScheme();
-    }
-  }
-
-  private static class RecoveryExceptionStandardScheme extends org.apache.thrift.scheme.StandardScheme<RecoveryException> {
-
-    @Override
-    public void read(org.apache.thrift.protocol.TProtocol iprot, RecoveryException 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: // WHY
-            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
-              struct.why = iprot.readString();
-              struct.setWhyIsSet(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();
-    }
-
-    @Override
-    public void write(org.apache.thrift.protocol.TProtocol oprot, RecoveryException struct) throws org.apache.thrift.TException {
-      struct.validate();
-
-      oprot.writeStructBegin(STRUCT_DESC);
-      if (struct.why != null) {
-        oprot.writeFieldBegin(WHY_FIELD_DESC);
-        oprot.writeString(struct.why);
-        oprot.writeFieldEnd();
-      }
-      oprot.writeFieldStop();
-      oprot.writeStructEnd();
-    }
-
-  }
-
-  private static class RecoveryExceptionTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-    @Override
-    public RecoveryExceptionTupleScheme getScheme() {
-      return new RecoveryExceptionTupleScheme();
-    }
-  }
-
-  private static class RecoveryExceptionTupleScheme extends org.apache.thrift.scheme.TupleScheme<RecoveryException> {
-
-    @Override
-    public void write(org.apache.thrift.protocol.TProtocol prot, RecoveryException 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.isSetWhy()) {
-        optionals.set(0);
-      }
-      oprot.writeBitSet(optionals, 1);
-      if (struct.isSetWhy()) {
-        oprot.writeString(struct.why);
-      }
-    }
-
-    @Override
-    public void read(org.apache.thrift.protocol.TProtocol prot, RecoveryException 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.why = iprot.readString();
-        struct.setWhyIsSet(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();
-  }
-  private static void unusedMethod() {}
-}
-
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/master/thrift/RecoveryStatus.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/RecoveryStatus.java
similarity index 99%
rename from core/src/main/thrift-gen-java/org/apache/accumulo/core/master/thrift/RecoveryStatus.java
rename to core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/RecoveryStatus.java
index 9372d5f..021911c 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/master/thrift/RecoveryStatus.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/RecoveryStatus.java
@@ -22,7 +22,7 @@
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
  */
-package org.apache.accumulo.core.master.thrift;
+package org.apache.accumulo.core.manager.thrift;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
 public class RecoveryStatus implements org.apache.thrift.TBase<RecoveryStatus, RecoveryStatus._Fields>, java.io.Serializable, Cloneable, Comparable<RecoveryStatus> {
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/master/thrift/TableInfo.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/TableInfo.java
similarity index 99%
rename from core/src/main/thrift-gen-java/org/apache/accumulo/core/master/thrift/TableInfo.java
rename to core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/TableInfo.java
index c9fb233..846adee 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/master/thrift/TableInfo.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/TableInfo.java
@@ -22,7 +22,7 @@
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
  */
-package org.apache.accumulo.core.master.thrift;
+package org.apache.accumulo.core.manager.thrift;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
 public class TableInfo implements org.apache.thrift.TBase<TableInfo, TableInfo._Fields>, java.io.Serializable, Cloneable, Comparable<TableInfo> {
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/TabletLoadState.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/TabletLoadState.java
index aa83733..8f06a49 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/TabletLoadState.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/TabletLoadState.java
@@ -30,8 +30,7 @@
   LOAD_FAILURE(1),
   UNLOADED(2),
   UNLOAD_FAILURE_NOT_SERVING(3),
-  UNLOAD_ERROR(4),
-  CHOPPED(5);
+  UNLOAD_ERROR(4);
 
   private final int value;
 
@@ -64,8 +63,6 @@
         return UNLOAD_FAILURE_NOT_SERVING;
       case 4:
         return UNLOAD_ERROR;
-      case 5:
-        return CHOPPED;
       default:
         return null;
     }
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/master/thrift/TabletServerStatus.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/TabletServerStatus.java
similarity index 96%
rename from core/src/main/thrift-gen-java/org/apache/accumulo/core/master/thrift/TabletServerStatus.java
rename to core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/TabletServerStatus.java
index c6261fc..36bf831 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/master/thrift/TabletServerStatus.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/TabletServerStatus.java
@@ -22,7 +22,7 @@
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
  */
-package org.apache.accumulo.core.master.thrift;
+package org.apache.accumulo.core.manager.thrift;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
 public class TabletServerStatus implements org.apache.thrift.TBase<TabletServerStatus, TabletServerStatus._Fields>, java.io.Serializable, Cloneable, Comparable<TabletServerStatus> {
@@ -1557,16 +1557,16 @@
           case 1: // TABLE_MAP
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map0 = iprot.readMapBegin();
-                struct.tableMap = new java.util.HashMap<java.lang.String,TableInfo>(2*_map0.size);
-                @org.apache.thrift.annotation.Nullable java.lang.String _key1;
-                @org.apache.thrift.annotation.Nullable TableInfo _val2;
-                for (int _i3 = 0; _i3 < _map0.size; ++_i3)
+                org.apache.thrift.protocol.TMap _map8 = iprot.readMapBegin();
+                struct.tableMap = new java.util.HashMap<java.lang.String,TableInfo>(2*_map8.size);
+                @org.apache.thrift.annotation.Nullable java.lang.String _key9;
+                @org.apache.thrift.annotation.Nullable TableInfo _val10;
+                for (int _i11 = 0; _i11 < _map8.size; ++_i11)
                 {
-                  _key1 = iprot.readString();
-                  _val2 = new TableInfo();
-                  _val2.read(iprot);
-                  struct.tableMap.put(_key1, _val2);
+                  _key9 = iprot.readString();
+                  _val10 = new TableInfo();
+                  _val10.read(iprot);
+                  struct.tableMap.put(_key9, _val10);
                 }
                 iprot.readMapEnd();
               }
@@ -1650,14 +1650,14 @@
           case 14: // LOG_SORTS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list4 = iprot.readListBegin();
-                struct.logSorts = new java.util.ArrayList<RecoveryStatus>(_list4.size);
-                @org.apache.thrift.annotation.Nullable RecoveryStatus _elem5;
-                for (int _i6 = 0; _i6 < _list4.size; ++_i6)
+                org.apache.thrift.protocol.TList _list12 = iprot.readListBegin();
+                struct.logSorts = new java.util.ArrayList<RecoveryStatus>(_list12.size);
+                @org.apache.thrift.annotation.Nullable RecoveryStatus _elem13;
+                for (int _i14 = 0; _i14 < _list12.size; ++_i14)
                 {
-                  _elem5 = new RecoveryStatus();
-                  _elem5.read(iprot);
-                  struct.logSorts.add(_elem5);
+                  _elem13 = new RecoveryStatus();
+                  _elem13.read(iprot);
+                  struct.logSorts.add(_elem13);
                 }
                 iprot.readListEnd();
               }
@@ -1685,14 +1685,14 @@
           case 17: // BULK_IMPORTS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list7 = iprot.readListBegin();
-                struct.bulkImports = new java.util.ArrayList<BulkImportStatus>(_list7.size);
-                @org.apache.thrift.annotation.Nullable BulkImportStatus _elem8;
-                for (int _i9 = 0; _i9 < _list7.size; ++_i9)
+                org.apache.thrift.protocol.TList _list15 = iprot.readListBegin();
+                struct.bulkImports = new java.util.ArrayList<BulkImportStatus>(_list15.size);
+                @org.apache.thrift.annotation.Nullable BulkImportStatus _elem16;
+                for (int _i17 = 0; _i17 < _list15.size; ++_i17)
                 {
-                  _elem8 = new BulkImportStatus();
-                  _elem8.read(iprot);
-                  struct.bulkImports.add(_elem8);
+                  _elem16 = new BulkImportStatus();
+                  _elem16.read(iprot);
+                  struct.bulkImports.add(_elem16);
                 }
                 iprot.readListEnd();
               }
@@ -1737,10 +1737,10 @@
         oprot.writeFieldBegin(TABLE_MAP_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.tableMap.size()));
-          for (java.util.Map.Entry<java.lang.String, TableInfo> _iter10 : struct.tableMap.entrySet())
+          for (java.util.Map.Entry<java.lang.String, TableInfo> _iter18 : struct.tableMap.entrySet())
           {
-            oprot.writeString(_iter10.getKey());
-            _iter10.getValue().write(oprot);
+            oprot.writeString(_iter18.getKey());
+            _iter18.getValue().write(oprot);
           }
           oprot.writeMapEnd();
         }
@@ -1779,9 +1779,9 @@
         oprot.writeFieldBegin(LOG_SORTS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.logSorts.size()));
-          for (RecoveryStatus _iter11 : struct.logSorts)
+          for (RecoveryStatus _iter19 : struct.logSorts)
           {
-            _iter11.write(oprot);
+            _iter19.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -1797,9 +1797,9 @@
         oprot.writeFieldBegin(BULK_IMPORTS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.bulkImports.size()));
-          for (BulkImportStatus _iter12 : struct.bulkImports)
+          for (BulkImportStatus _iter20 : struct.bulkImports)
           {
-            _iter12.write(oprot);
+            _iter20.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -1884,10 +1884,10 @@
       if (struct.isSetTableMap()) {
         {
           oprot.writeI32(struct.tableMap.size());
-          for (java.util.Map.Entry<java.lang.String, TableInfo> _iter13 : struct.tableMap.entrySet())
+          for (java.util.Map.Entry<java.lang.String, TableInfo> _iter21 : struct.tableMap.entrySet())
           {
-            oprot.writeString(_iter13.getKey());
-            _iter13.getValue().write(oprot);
+            oprot.writeString(_iter21.getKey());
+            _iter21.getValue().write(oprot);
           }
         }
       }
@@ -1921,9 +1921,9 @@
       if (struct.isSetLogSorts()) {
         {
           oprot.writeI32(struct.logSorts.size());
-          for (RecoveryStatus _iter14 : struct.logSorts)
+          for (RecoveryStatus _iter22 : struct.logSorts)
           {
-            _iter14.write(oprot);
+            _iter22.write(oprot);
           }
         }
       }
@@ -1936,9 +1936,9 @@
       if (struct.isSetBulkImports()) {
         {
           oprot.writeI32(struct.bulkImports.size());
-          for (BulkImportStatus _iter15 : struct.bulkImports)
+          for (BulkImportStatus _iter23 : struct.bulkImports)
           {
-            _iter15.write(oprot);
+            _iter23.write(oprot);
           }
         }
       }
@@ -1956,16 +1956,16 @@
       java.util.BitSet incoming = iprot.readBitSet(16);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TMap _map16 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT); 
-          struct.tableMap = new java.util.HashMap<java.lang.String,TableInfo>(2*_map16.size);
-          @org.apache.thrift.annotation.Nullable java.lang.String _key17;
-          @org.apache.thrift.annotation.Nullable TableInfo _val18;
-          for (int _i19 = 0; _i19 < _map16.size; ++_i19)
+          org.apache.thrift.protocol.TMap _map24 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT); 
+          struct.tableMap = new java.util.HashMap<java.lang.String,TableInfo>(2*_map24.size);
+          @org.apache.thrift.annotation.Nullable java.lang.String _key25;
+          @org.apache.thrift.annotation.Nullable TableInfo _val26;
+          for (int _i27 = 0; _i27 < _map24.size; ++_i27)
           {
-            _key17 = iprot.readString();
-            _val18 = new TableInfo();
-            _val18.read(iprot);
-            struct.tableMap.put(_key17, _val18);
+            _key25 = iprot.readString();
+            _val26 = new TableInfo();
+            _val26.read(iprot);
+            struct.tableMap.put(_key25, _val26);
           }
         }
         struct.setTableMapIsSet(true);
@@ -2008,14 +2008,14 @@
       }
       if (incoming.get(10)) {
         {
-          org.apache.thrift.protocol.TList _list20 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
-          struct.logSorts = new java.util.ArrayList<RecoveryStatus>(_list20.size);
-          @org.apache.thrift.annotation.Nullable RecoveryStatus _elem21;
-          for (int _i22 = 0; _i22 < _list20.size; ++_i22)
+          org.apache.thrift.protocol.TList _list28 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+          struct.logSorts = new java.util.ArrayList<RecoveryStatus>(_list28.size);
+          @org.apache.thrift.annotation.Nullable RecoveryStatus _elem29;
+          for (int _i30 = 0; _i30 < _list28.size; ++_i30)
           {
-            _elem21 = new RecoveryStatus();
-            _elem21.read(iprot);
-            struct.logSorts.add(_elem21);
+            _elem29 = new RecoveryStatus();
+            _elem29.read(iprot);
+            struct.logSorts.add(_elem29);
           }
         }
         struct.setLogSortsIsSet(true);
@@ -2030,14 +2030,14 @@
       }
       if (incoming.get(13)) {
         {
-          org.apache.thrift.protocol.TList _list23 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
-          struct.bulkImports = new java.util.ArrayList<BulkImportStatus>(_list23.size);
-          @org.apache.thrift.annotation.Nullable BulkImportStatus _elem24;
-          for (int _i25 = 0; _i25 < _list23.size; ++_i25)
+          org.apache.thrift.protocol.TList _list31 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+          struct.bulkImports = new java.util.ArrayList<BulkImportStatus>(_list31.size);
+          @org.apache.thrift.annotation.Nullable BulkImportStatus _elem32;
+          for (int _i33 = 0; _i33 < _list31.size; ++_i33)
           {
-            _elem24 = new BulkImportStatus();
-            _elem24.read(iprot);
-            struct.bulkImports.add(_elem24);
+            _elem32 = new BulkImportStatus();
+            _elem32.read(iprot);
+            struct.bulkImports.add(_elem32);
           }
         }
         struct.setBulkImportsIsSet(true);
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/replication/thrift/KeyValues.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/replication/thrift/KeyValues.java
deleted file mode 100644
index b700d31..0000000
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/replication/thrift/KeyValues.java
+++ /dev/null
@@ -1,461 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/**
- * Autogenerated by Thrift Compiler (0.17.0)
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- *  @generated
- */
-package org.apache.accumulo.core.replication.thrift;
-
-@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-public class KeyValues implements org.apache.thrift.TBase<KeyValues, KeyValues._Fields>, java.io.Serializable, Cloneable, Comparable<KeyValues> {
-  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("KeyValues");
-
-  private static final org.apache.thrift.protocol.TField KEY_VALUES_FIELD_DESC = new org.apache.thrift.protocol.TField("keyValues", org.apache.thrift.protocol.TType.LIST, (short)1);
-
-  private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new KeyValuesStandardSchemeFactory();
-  private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new KeyValuesTupleSchemeFactory();
-
-  public @org.apache.thrift.annotation.Nullable java.util.List<org.apache.accumulo.core.dataImpl.thrift.TKeyValue> keyValues; // 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 {
-    KEY_VALUES((short)1, "keyValues");
-
-    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: // KEY_VALUES
-          return KEY_VALUES;
-        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;
-    }
-
-    @Override
-    public short getThriftFieldId() {
-      return _thriftId;
-    }
-
-    @Override
-    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.KEY_VALUES, new org.apache.thrift.meta_data.FieldMetaData("keyValues", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
-            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.dataImpl.thrift.TKeyValue.class))));
-    metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(KeyValues.class, metaDataMap);
-  }
-
-  public KeyValues() {
-  }
-
-  public KeyValues(
-    java.util.List<org.apache.accumulo.core.dataImpl.thrift.TKeyValue> keyValues)
-  {
-    this();
-    this.keyValues = keyValues;
-  }
-
-  /**
-   * Performs a deep copy on <i>other</i>.
-   */
-  public KeyValues(KeyValues other) {
-    if (other.isSetKeyValues()) {
-      java.util.List<org.apache.accumulo.core.dataImpl.thrift.TKeyValue> __this__keyValues = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TKeyValue>(other.keyValues.size());
-      for (org.apache.accumulo.core.dataImpl.thrift.TKeyValue other_element : other.keyValues) {
-        __this__keyValues.add(new org.apache.accumulo.core.dataImpl.thrift.TKeyValue(other_element));
-      }
-      this.keyValues = __this__keyValues;
-    }
-  }
-
-  @Override
-  public KeyValues deepCopy() {
-    return new KeyValues(this);
-  }
-
-  @Override
-  public void clear() {
-    this.keyValues = null;
-  }
-
-  public int getKeyValuesSize() {
-    return (this.keyValues == null) ? 0 : this.keyValues.size();
-  }
-
-  @org.apache.thrift.annotation.Nullable
-  public java.util.Iterator<org.apache.accumulo.core.dataImpl.thrift.TKeyValue> getKeyValuesIterator() {
-    return (this.keyValues == null) ? null : this.keyValues.iterator();
-  }
-
-  public void addToKeyValues(org.apache.accumulo.core.dataImpl.thrift.TKeyValue elem) {
-    if (this.keyValues == null) {
-      this.keyValues = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TKeyValue>();
-    }
-    this.keyValues.add(elem);
-  }
-
-  @org.apache.thrift.annotation.Nullable
-  public java.util.List<org.apache.accumulo.core.dataImpl.thrift.TKeyValue> getKeyValues() {
-    return this.keyValues;
-  }
-
-  public KeyValues setKeyValues(@org.apache.thrift.annotation.Nullable java.util.List<org.apache.accumulo.core.dataImpl.thrift.TKeyValue> keyValues) {
-    this.keyValues = keyValues;
-    return this;
-  }
-
-  public void unsetKeyValues() {
-    this.keyValues = null;
-  }
-
-  /** Returns true if field keyValues is set (has been assigned a value) and false otherwise */
-  public boolean isSetKeyValues() {
-    return this.keyValues != null;
-  }
-
-  public void setKeyValuesIsSet(boolean value) {
-    if (!value) {
-      this.keyValues = null;
-    }
-  }
-
-  @Override
-  public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
-    switch (field) {
-    case KEY_VALUES:
-      if (value == null) {
-        unsetKeyValues();
-      } else {
-        setKeyValues((java.util.List<org.apache.accumulo.core.dataImpl.thrift.TKeyValue>)value);
-      }
-      break;
-
-    }
-  }
-
-  @org.apache.thrift.annotation.Nullable
-  @Override
-  public java.lang.Object getFieldValue(_Fields field) {
-    switch (field) {
-    case KEY_VALUES:
-      return getKeyValues();
-
-    }
-    throw new java.lang.IllegalStateException();
-  }
-
-  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
-  @Override
-  public boolean isSet(_Fields field) {
-    if (field == null) {
-      throw new java.lang.IllegalArgumentException();
-    }
-
-    switch (field) {
-    case KEY_VALUES:
-      return isSetKeyValues();
-    }
-    throw new java.lang.IllegalStateException();
-  }
-
-  @Override
-  public boolean equals(java.lang.Object that) {
-    if (that instanceof KeyValues)
-      return this.equals((KeyValues)that);
-    return false;
-  }
-
-  public boolean equals(KeyValues that) {
-    if (that == null)
-      return false;
-    if (this == that)
-      return true;
-
-    boolean this_present_keyValues = true && this.isSetKeyValues();
-    boolean that_present_keyValues = true && that.isSetKeyValues();
-    if (this_present_keyValues || that_present_keyValues) {
-      if (!(this_present_keyValues && that_present_keyValues))
-        return false;
-      if (!this.keyValues.equals(that.keyValues))
-        return false;
-    }
-
-    return true;
-  }
-
-  @Override
-  public int hashCode() {
-    int hashCode = 1;
-
-    hashCode = hashCode * 8191 + ((isSetKeyValues()) ? 131071 : 524287);
-    if (isSetKeyValues())
-      hashCode = hashCode * 8191 + keyValues.hashCode();
-
-    return hashCode;
-  }
-
-  @Override
-  public int compareTo(KeyValues other) {
-    if (!getClass().equals(other.getClass())) {
-      return getClass().getName().compareTo(other.getClass().getName());
-    }
-
-    int lastComparison = 0;
-
-    lastComparison = java.lang.Boolean.compare(isSetKeyValues(), other.isSetKeyValues());
-    if (lastComparison != 0) {
-      return lastComparison;
-    }
-    if (isSetKeyValues()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.keyValues, other.keyValues);
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-    }
-    return 0;
-  }
-
-  @org.apache.thrift.annotation.Nullable
-  @Override
-  public _Fields fieldForId(int fieldId) {
-    return _Fields.findByThriftId(fieldId);
-  }
-
-  @Override
-  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
-    scheme(iprot).read(iprot, this);
-  }
-
-  @Override
-  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("KeyValues(");
-    boolean first = true;
-
-    sb.append("keyValues:");
-    if (this.keyValues == null) {
-      sb.append("null");
-    } else {
-      sb.append(this.keyValues);
-    }
-    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 KeyValuesStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-    @Override
-    public KeyValuesStandardScheme getScheme() {
-      return new KeyValuesStandardScheme();
-    }
-  }
-
-  private static class KeyValuesStandardScheme extends org.apache.thrift.scheme.StandardScheme<KeyValues> {
-
-    @Override
-    public void read(org.apache.thrift.protocol.TProtocol iprot, KeyValues 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: // KEY_VALUES
-            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
-              {
-                org.apache.thrift.protocol.TList _list8 = iprot.readListBegin();
-                struct.keyValues = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TKeyValue>(_list8.size);
-                @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyValue _elem9;
-                for (int _i10 = 0; _i10 < _list8.size; ++_i10)
-                {
-                  _elem9 = new org.apache.accumulo.core.dataImpl.thrift.TKeyValue();
-                  _elem9.read(iprot);
-                  struct.keyValues.add(_elem9);
-                }
-                iprot.readListEnd();
-              }
-              struct.setKeyValuesIsSet(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();
-    }
-
-    @Override
-    public void write(org.apache.thrift.protocol.TProtocol oprot, KeyValues struct) throws org.apache.thrift.TException {
-      struct.validate();
-
-      oprot.writeStructBegin(STRUCT_DESC);
-      if (struct.keyValues != null) {
-        oprot.writeFieldBegin(KEY_VALUES_FIELD_DESC);
-        {
-          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.keyValues.size()));
-          for (org.apache.accumulo.core.dataImpl.thrift.TKeyValue _iter11 : struct.keyValues)
-          {
-            _iter11.write(oprot);
-          }
-          oprot.writeListEnd();
-        }
-        oprot.writeFieldEnd();
-      }
-      oprot.writeFieldStop();
-      oprot.writeStructEnd();
-    }
-
-  }
-
-  private static class KeyValuesTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-    @Override
-    public KeyValuesTupleScheme getScheme() {
-      return new KeyValuesTupleScheme();
-    }
-  }
-
-  private static class KeyValuesTupleScheme extends org.apache.thrift.scheme.TupleScheme<KeyValues> {
-
-    @Override
-    public void write(org.apache.thrift.protocol.TProtocol prot, KeyValues 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.isSetKeyValues()) {
-        optionals.set(0);
-      }
-      oprot.writeBitSet(optionals, 1);
-      if (struct.isSetKeyValues()) {
-        {
-          oprot.writeI32(struct.keyValues.size());
-          for (org.apache.accumulo.core.dataImpl.thrift.TKeyValue _iter12 : struct.keyValues)
-          {
-            _iter12.write(oprot);
-          }
-        }
-      }
-    }
-
-    @Override
-    public void read(org.apache.thrift.protocol.TProtocol prot, KeyValues 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.TList _list13 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
-          struct.keyValues = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TKeyValue>(_list13.size);
-          @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyValue _elem14;
-          for (int _i15 = 0; _i15 < _list13.size; ++_i15)
-          {
-            _elem14 = new org.apache.accumulo.core.dataImpl.thrift.TKeyValue();
-            _elem14.read(iprot);
-            struct.keyValues.add(_elem14);
-          }
-        }
-        struct.setKeyValuesIsSet(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();
-  }
-  private static void unusedMethod() {}
-}
-
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/replication/thrift/RemoteReplicationErrorCode.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/replication/thrift/RemoteReplicationErrorCode.java
deleted file mode 100644
index 9d9d810..0000000
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/replication/thrift/RemoteReplicationErrorCode.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/**
- * Autogenerated by Thrift Compiler (0.17.0)
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- *  @generated
- */
-package org.apache.accumulo.core.replication.thrift;
-
-
-public enum RemoteReplicationErrorCode implements org.apache.thrift.TEnum {
-  COULD_NOT_DESERIALIZE(0),
-  COULD_NOT_APPLY(1),
-  TABLE_DOES_NOT_EXIST(2),
-  CANNOT_AUTHENTICATE(3),
-  CANNOT_INSTANTIATE_REPLAYER(4);
-
-  private final int value;
-
-  private RemoteReplicationErrorCode(int value) {
-    this.value = value;
-  }
-
-  /**
-   * Get the integer value of this enum value, as defined in the Thrift IDL.
-   */
-  @Override
-  public int getValue() {
-    return value;
-  }
-
-  /**
-   * Find a the enum type by its integer value, as defined in the Thrift IDL.
-   * @return null if the value is not found.
-   */
-  @org.apache.thrift.annotation.Nullable
-  public static RemoteReplicationErrorCode findByValue(int value) { 
-    switch (value) {
-      case 0:
-        return COULD_NOT_DESERIALIZE;
-      case 1:
-        return COULD_NOT_APPLY;
-      case 2:
-        return TABLE_DOES_NOT_EXIST;
-      case 3:
-        return CANNOT_AUTHENTICATE;
-      case 4:
-        return CANNOT_INSTANTIATE_REPLAYER;
-      default:
-        return null;
-    }
-  }
-}
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/replication/thrift/RemoteReplicationException.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/replication/thrift/RemoteReplicationException.java
deleted file mode 100644
index 8c250e0..0000000
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/replication/thrift/RemoteReplicationException.java
+++ /dev/null
@@ -1,527 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/**
- * Autogenerated by Thrift Compiler (0.17.0)
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- *  @generated
- */
-package org.apache.accumulo.core.replication.thrift;
-
-@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-public class RemoteReplicationException extends org.apache.thrift.TException implements org.apache.thrift.TBase<RemoteReplicationException, RemoteReplicationException._Fields>, java.io.Serializable, Cloneable, Comparable<RemoteReplicationException> {
-  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("RemoteReplicationException");
-
-  private static final org.apache.thrift.protocol.TField CODE_FIELD_DESC = new org.apache.thrift.protocol.TField("code", org.apache.thrift.protocol.TType.I32, (short)1);
-  private static final org.apache.thrift.protocol.TField REASON_FIELD_DESC = new org.apache.thrift.protocol.TField("reason", org.apache.thrift.protocol.TType.STRING, (short)2);
-
-  private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new RemoteReplicationExceptionStandardSchemeFactory();
-  private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new RemoteReplicationExceptionTupleSchemeFactory();
-
-  /**
-   * 
-   * @see RemoteReplicationErrorCode
-   */
-  public @org.apache.thrift.annotation.Nullable RemoteReplicationErrorCode code; // required
-  public @org.apache.thrift.annotation.Nullable java.lang.String reason; // 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 {
-    /**
-     * 
-     * @see RemoteReplicationErrorCode
-     */
-    CODE((short)1, "code"),
-    REASON((short)2, "reason");
-
-    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: // CODE
-          return CODE;
-        case 2: // REASON
-          return REASON;
-        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;
-    }
-
-    @Override
-    public short getThriftFieldId() {
-      return _thriftId;
-    }
-
-    @Override
-    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.CODE, new org.apache.thrift.meta_data.FieldMetaData("code", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-        new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, RemoteReplicationErrorCode.class)));
-    tmpMap.put(_Fields.REASON, new org.apache.thrift.meta_data.FieldMetaData("reason", 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(RemoteReplicationException.class, metaDataMap);
-  }
-
-  public RemoteReplicationException() {
-  }
-
-  public RemoteReplicationException(
-    RemoteReplicationErrorCode code,
-    java.lang.String reason)
-  {
-    this();
-    this.code = code;
-    this.reason = reason;
-  }
-
-  /**
-   * Performs a deep copy on <i>other</i>.
-   */
-  public RemoteReplicationException(RemoteReplicationException other) {
-    if (other.isSetCode()) {
-      this.code = other.code;
-    }
-    if (other.isSetReason()) {
-      this.reason = other.reason;
-    }
-  }
-
-  @Override
-  public RemoteReplicationException deepCopy() {
-    return new RemoteReplicationException(this);
-  }
-
-  @Override
-  public void clear() {
-    this.code = null;
-    this.reason = null;
-  }
-
-  /**
-   * 
-   * @see RemoteReplicationErrorCode
-   */
-  @org.apache.thrift.annotation.Nullable
-  public RemoteReplicationErrorCode getCode() {
-    return this.code;
-  }
-
-  /**
-   * 
-   * @see RemoteReplicationErrorCode
-   */
-  public RemoteReplicationException setCode(@org.apache.thrift.annotation.Nullable RemoteReplicationErrorCode code) {
-    this.code = code;
-    return this;
-  }
-
-  public void unsetCode() {
-    this.code = null;
-  }
-
-  /** Returns true if field code is set (has been assigned a value) and false otherwise */
-  public boolean isSetCode() {
-    return this.code != null;
-  }
-
-  public void setCodeIsSet(boolean value) {
-    if (!value) {
-      this.code = null;
-    }
-  }
-
-  @org.apache.thrift.annotation.Nullable
-  public java.lang.String getReason() {
-    return this.reason;
-  }
-
-  public RemoteReplicationException setReason(@org.apache.thrift.annotation.Nullable java.lang.String reason) {
-    this.reason = reason;
-    return this;
-  }
-
-  public void unsetReason() {
-    this.reason = null;
-  }
-
-  /** Returns true if field reason is set (has been assigned a value) and false otherwise */
-  public boolean isSetReason() {
-    return this.reason != null;
-  }
-
-  public void setReasonIsSet(boolean value) {
-    if (!value) {
-      this.reason = null;
-    }
-  }
-
-  @Override
-  public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
-    switch (field) {
-    case CODE:
-      if (value == null) {
-        unsetCode();
-      } else {
-        setCode((RemoteReplicationErrorCode)value);
-      }
-      break;
-
-    case REASON:
-      if (value == null) {
-        unsetReason();
-      } else {
-        setReason((java.lang.String)value);
-      }
-      break;
-
-    }
-  }
-
-  @org.apache.thrift.annotation.Nullable
-  @Override
-  public java.lang.Object getFieldValue(_Fields field) {
-    switch (field) {
-    case CODE:
-      return getCode();
-
-    case REASON:
-      return getReason();
-
-    }
-    throw new java.lang.IllegalStateException();
-  }
-
-  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
-  @Override
-  public boolean isSet(_Fields field) {
-    if (field == null) {
-      throw new java.lang.IllegalArgumentException();
-    }
-
-    switch (field) {
-    case CODE:
-      return isSetCode();
-    case REASON:
-      return isSetReason();
-    }
-    throw new java.lang.IllegalStateException();
-  }
-
-  @Override
-  public boolean equals(java.lang.Object that) {
-    if (that instanceof RemoteReplicationException)
-      return this.equals((RemoteReplicationException)that);
-    return false;
-  }
-
-  public boolean equals(RemoteReplicationException that) {
-    if (that == null)
-      return false;
-    if (this == that)
-      return true;
-
-    boolean this_present_code = true && this.isSetCode();
-    boolean that_present_code = true && that.isSetCode();
-    if (this_present_code || that_present_code) {
-      if (!(this_present_code && that_present_code))
-        return false;
-      if (!this.code.equals(that.code))
-        return false;
-    }
-
-    boolean this_present_reason = true && this.isSetReason();
-    boolean that_present_reason = true && that.isSetReason();
-    if (this_present_reason || that_present_reason) {
-      if (!(this_present_reason && that_present_reason))
-        return false;
-      if (!this.reason.equals(that.reason))
-        return false;
-    }
-
-    return true;
-  }
-
-  @Override
-  public int hashCode() {
-    int hashCode = 1;
-
-    hashCode = hashCode * 8191 + ((isSetCode()) ? 131071 : 524287);
-    if (isSetCode())
-      hashCode = hashCode * 8191 + code.getValue();
-
-    hashCode = hashCode * 8191 + ((isSetReason()) ? 131071 : 524287);
-    if (isSetReason())
-      hashCode = hashCode * 8191 + reason.hashCode();
-
-    return hashCode;
-  }
-
-  @Override
-  public int compareTo(RemoteReplicationException other) {
-    if (!getClass().equals(other.getClass())) {
-      return getClass().getName().compareTo(other.getClass().getName());
-    }
-
-    int lastComparison = 0;
-
-    lastComparison = java.lang.Boolean.compare(isSetCode(), other.isSetCode());
-    if (lastComparison != 0) {
-      return lastComparison;
-    }
-    if (isSetCode()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.code, other.code);
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-    }
-    lastComparison = java.lang.Boolean.compare(isSetReason(), other.isSetReason());
-    if (lastComparison != 0) {
-      return lastComparison;
-    }
-    if (isSetReason()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.reason, other.reason);
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-    }
-    return 0;
-  }
-
-  @org.apache.thrift.annotation.Nullable
-  @Override
-  public _Fields fieldForId(int fieldId) {
-    return _Fields.findByThriftId(fieldId);
-  }
-
-  @Override
-  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
-    scheme(iprot).read(iprot, this);
-  }
-
-  @Override
-  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("RemoteReplicationException(");
-    boolean first = true;
-
-    sb.append("code:");
-    if (this.code == null) {
-      sb.append("null");
-    } else {
-      sb.append(this.code);
-    }
-    first = false;
-    if (!first) sb.append(", ");
-    sb.append("reason:");
-    if (this.reason == null) {
-      sb.append("null");
-    } else {
-      sb.append(this.reason);
-    }
-    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 RemoteReplicationExceptionStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-    @Override
-    public RemoteReplicationExceptionStandardScheme getScheme() {
-      return new RemoteReplicationExceptionStandardScheme();
-    }
-  }
-
-  private static class RemoteReplicationExceptionStandardScheme extends org.apache.thrift.scheme.StandardScheme<RemoteReplicationException> {
-
-    @Override
-    public void read(org.apache.thrift.protocol.TProtocol iprot, RemoteReplicationException 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: // CODE
-            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
-              struct.code = org.apache.accumulo.core.replication.thrift.RemoteReplicationErrorCode.findByValue(iprot.readI32());
-              struct.setCodeIsSet(true);
-            } else { 
-              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-            }
-            break;
-          case 2: // REASON
-            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
-              struct.reason = iprot.readString();
-              struct.setReasonIsSet(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();
-    }
-
-    @Override
-    public void write(org.apache.thrift.protocol.TProtocol oprot, RemoteReplicationException struct) throws org.apache.thrift.TException {
-      struct.validate();
-
-      oprot.writeStructBegin(STRUCT_DESC);
-      if (struct.code != null) {
-        oprot.writeFieldBegin(CODE_FIELD_DESC);
-        oprot.writeI32(struct.code.getValue());
-        oprot.writeFieldEnd();
-      }
-      if (struct.reason != null) {
-        oprot.writeFieldBegin(REASON_FIELD_DESC);
-        oprot.writeString(struct.reason);
-        oprot.writeFieldEnd();
-      }
-      oprot.writeFieldStop();
-      oprot.writeStructEnd();
-    }
-
-  }
-
-  private static class RemoteReplicationExceptionTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-    @Override
-    public RemoteReplicationExceptionTupleScheme getScheme() {
-      return new RemoteReplicationExceptionTupleScheme();
-    }
-  }
-
-  private static class RemoteReplicationExceptionTupleScheme extends org.apache.thrift.scheme.TupleScheme<RemoteReplicationException> {
-
-    @Override
-    public void write(org.apache.thrift.protocol.TProtocol prot, RemoteReplicationException 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.isSetCode()) {
-        optionals.set(0);
-      }
-      if (struct.isSetReason()) {
-        optionals.set(1);
-      }
-      oprot.writeBitSet(optionals, 2);
-      if (struct.isSetCode()) {
-        oprot.writeI32(struct.code.getValue());
-      }
-      if (struct.isSetReason()) {
-        oprot.writeString(struct.reason);
-      }
-    }
-
-    @Override
-    public void read(org.apache.thrift.protocol.TProtocol prot, RemoteReplicationException 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.code = org.apache.accumulo.core.replication.thrift.RemoteReplicationErrorCode.findByValue(iprot.readI32());
-        struct.setCodeIsSet(true);
-      }
-      if (incoming.get(1)) {
-        struct.reason = iprot.readString();
-        struct.setReasonIsSet(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();
-  }
-  private static void unusedMethod() {}
-}
-
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/replication/thrift/ReplicationCoordinator.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/replication/thrift/ReplicationCoordinator.java
deleted file mode 100644
index b4e79d6..0000000
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/replication/thrift/ReplicationCoordinator.java
+++ /dev/null
@@ -1,1263 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/**
- * Autogenerated by Thrift Compiler (0.17.0)
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- *  @generated
- */
-package org.apache.accumulo.core.replication.thrift;
-
-@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-public class ReplicationCoordinator {
-
-  public interface Iface {
-
-    public java.lang.String getServicerAddress(java.lang.String remoteTableId, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws ReplicationCoordinatorException, org.apache.thrift.TException;
-
-  }
-
-  public interface AsyncIface {
-
-    public void getServicerAddress(java.lang.String remoteTableId, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler) throws org.apache.thrift.TException;
-
-  }
-
-  public static class Client extends org.apache.thrift.TServiceClient implements Iface {
-    public static class Factory implements org.apache.thrift.TServiceClientFactory<Client> {
-      public Factory() {}
-      @Override
-      public Client getClient(org.apache.thrift.protocol.TProtocol prot) {
-        return new Client(prot);
-      }
-      @Override
-      public Client getClient(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TProtocol oprot) {
-        return new Client(iprot, oprot);
-      }
-    }
-
-    public Client(org.apache.thrift.protocol.TProtocol prot)
-    {
-      super(prot, prot);
-    }
-
-    public Client(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TProtocol oprot) {
-      super(iprot, oprot);
-    }
-
-    @Override
-    public java.lang.String getServicerAddress(java.lang.String remoteTableId, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws ReplicationCoordinatorException, org.apache.thrift.TException
-    {
-      send_getServicerAddress(remoteTableId, credentials);
-      return recv_getServicerAddress();
-    }
-
-    public void send_getServicerAddress(java.lang.String remoteTableId, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
-    {
-      getServicerAddress_args args = new getServicerAddress_args();
-      args.setRemoteTableId(remoteTableId);
-      args.setCredentials(credentials);
-      sendBase("getServicerAddress", args);
-    }
-
-    public java.lang.String recv_getServicerAddress() throws ReplicationCoordinatorException, org.apache.thrift.TException
-    {
-      getServicerAddress_result result = new getServicerAddress_result();
-      receiveBase(result, "getServicerAddress");
-      if (result.isSetSuccess()) {
-        return result.success;
-      }
-      if (result.e != null) {
-        throw result.e;
-      }
-      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getServicerAddress failed: unknown result");
-    }
-
-  }
-  public static class AsyncClient extends org.apache.thrift.async.TAsyncClient implements AsyncIface {
-    public static class Factory implements org.apache.thrift.async.TAsyncClientFactory<AsyncClient> {
-      private org.apache.thrift.async.TAsyncClientManager clientManager;
-      private org.apache.thrift.protocol.TProtocolFactory protocolFactory;
-      public Factory(org.apache.thrift.async.TAsyncClientManager clientManager, org.apache.thrift.protocol.TProtocolFactory protocolFactory) {
-        this.clientManager = clientManager;
-        this.protocolFactory = protocolFactory;
-      }
-    @Override
-      public AsyncClient getAsyncClient(org.apache.thrift.transport.TNonblockingTransport transport) {
-        return new AsyncClient(protocolFactory, clientManager, transport);
-      }
-    }
-
-    public AsyncClient(org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.async.TAsyncClientManager clientManager, org.apache.thrift.transport.TNonblockingTransport transport) {
-      super(protocolFactory, clientManager, transport);
-    }
-
-    @Override
-    public void getServicerAddress(java.lang.String remoteTableId, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler) throws org.apache.thrift.TException {
-      checkReady();
-      getServicerAddress_call method_call = new getServicerAddress_call(remoteTableId, credentials, resultHandler, this, ___protocolFactory, ___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
-    }
-
-    public static class getServicerAddress_call extends org.apache.thrift.async.TAsyncMethodCall<java.lang.String> {
-      private java.lang.String remoteTableId;
-      private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
-      public getServicerAddress_call(java.lang.String remoteTableId, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
-        super(client, protocolFactory, transport, resultHandler, false);
-        this.remoteTableId = remoteTableId;
-        this.credentials = credentials;
-      }
-
-      @Override
-      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getServicerAddress", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        getServicerAddress_args args = new getServicerAddress_args();
-        args.setRemoteTableId(remoteTableId);
-        args.setCredentials(credentials);
-        args.write(prot);
-        prot.writeMessageEnd();
-      }
-
-      @Override
-      public java.lang.String getResult() throws ReplicationCoordinatorException, 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_getServicerAddress();
-      }
-    }
-
-  }
-
-  public static class Processor<I extends Iface> extends org.apache.thrift.TBaseProcessor<I> implements org.apache.thrift.TProcessor {
-    private static final org.slf4j.Logger _LOGGER = org.slf4j.LoggerFactory.getLogger(Processor.class.getName());
-    public Processor(I iface) {
-      super(iface, getProcessMap(new java.util.HashMap<java.lang.String, org.apache.thrift.ProcessFunction<I, ? extends org.apache.thrift.TBase>>()));
-    }
-
-    protected Processor(I iface, java.util.Map<java.lang.String, org.apache.thrift.ProcessFunction<I, ? extends org.apache.thrift.TBase>> processMap) {
-      super(iface, getProcessMap(processMap));
-    }
-
-    private static <I extends Iface> java.util.Map<java.lang.String,  org.apache.thrift.ProcessFunction<I, ? extends org.apache.thrift.TBase>> getProcessMap(java.util.Map<java.lang.String, org.apache.thrift.ProcessFunction<I, ? extends  org.apache.thrift.TBase>> processMap) {
-      processMap.put("getServicerAddress", new getServicerAddress());
-      return processMap;
-    }
-
-    public static class getServicerAddress<I extends Iface> extends org.apache.thrift.ProcessFunction<I, getServicerAddress_args> {
-      public getServicerAddress() {
-        super("getServicerAddress");
-      }
-
-      @Override
-      public getServicerAddress_args getEmptyArgsInstance() {
-        return new getServicerAddress_args();
-      }
-
-      @Override
-      protected boolean isOneway() {
-        return false;
-      }
-
-      @Override
-      protected boolean rethrowUnhandledExceptions() {
-        return false;
-      }
-
-      @Override
-      public getServicerAddress_result getResult(I iface, getServicerAddress_args args) throws org.apache.thrift.TException {
-        getServicerAddress_result result = new getServicerAddress_result();
-        try {
-          result.success = iface.getServicerAddress(args.remoteTableId, args.credentials);
-        } catch (ReplicationCoordinatorException e) {
-          result.e = e;
-        }
-        return result;
-      }
-    }
-
-  }
-
-  public static class AsyncProcessor<I extends AsyncIface> extends org.apache.thrift.TBaseAsyncProcessor<I> {
-    private static final org.slf4j.Logger _LOGGER = org.slf4j.LoggerFactory.getLogger(AsyncProcessor.class.getName());
-    public AsyncProcessor(I iface) {
-      super(iface, getProcessMap(new java.util.HashMap<java.lang.String, org.apache.thrift.AsyncProcessFunction<I, ? extends org.apache.thrift.TBase, ?>>()));
-    }
-
-    protected AsyncProcessor(I iface, java.util.Map<java.lang.String,  org.apache.thrift.AsyncProcessFunction<I, ? extends  org.apache.thrift.TBase, ?>> processMap) {
-      super(iface, getProcessMap(processMap));
-    }
-
-    private static <I extends AsyncIface> java.util.Map<java.lang.String,  org.apache.thrift.AsyncProcessFunction<I, ? extends  org.apache.thrift.TBase,?>> getProcessMap(java.util.Map<java.lang.String,  org.apache.thrift.AsyncProcessFunction<I, ? extends  org.apache.thrift.TBase, ?>> processMap) {
-      processMap.put("getServicerAddress", new getServicerAddress());
-      return processMap;
-    }
-
-    public static class getServicerAddress<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getServicerAddress_args, java.lang.String> {
-      public getServicerAddress() {
-        super("getServicerAddress");
-      }
-
-      @Override
-      public getServicerAddress_args getEmptyArgsInstance() {
-        return new getServicerAddress_args();
-      }
-
-      @Override
-      public org.apache.thrift.async.AsyncMethodCallback<java.lang.String> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
-        final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new org.apache.thrift.async.AsyncMethodCallback<java.lang.String>() { 
-          @Override
-          public void onComplete(java.lang.String o) {
-            getServicerAddress_result result = new getServicerAddress_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);
-            }
-          }
-          @Override
-          public void onError(java.lang.Exception e) {
-            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
-            org.apache.thrift.TSerializable msg;
-            getServicerAddress_result result = new getServicerAddress_result();
-            if (e instanceof ReplicationCoordinatorException) {
-              result.e = (ReplicationCoordinatorException) e;
-              result.setEIsSet(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();
-            }
-          }
-        };
-      }
-
-      @Override
-      protected boolean isOneway() {
-        return false;
-      }
-
-      @Override
-      public void start(I iface, getServicerAddress_args args, org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler) throws org.apache.thrift.TException {
-        iface.getServicerAddress(args.remoteTableId, args.credentials,resultHandler);
-      }
-    }
-
-  }
-
-  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-  public static class getServicerAddress_args implements org.apache.thrift.TBase<getServicerAddress_args, getServicerAddress_args._Fields>, java.io.Serializable, Cloneable, Comparable<getServicerAddress_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getServicerAddress_args");
-
-    private static final org.apache.thrift.protocol.TField REMOTE_TABLE_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("remoteTableId", org.apache.thrift.protocol.TType.STRING, (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 getServicerAddress_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getServicerAddress_argsTupleSchemeFactory();
-
-    public @org.apache.thrift.annotation.Nullable java.lang.String remoteTableId; // 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 {
-      REMOTE_TABLE_ID((short)1, "remoteTableId"),
-      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: // REMOTE_TABLE_ID
-            return REMOTE_TABLE_ID;
-          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;
-      }
-
-      @Override
-      public short getThriftFieldId() {
-        return _thriftId;
-      }
-
-      @Override
-      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.REMOTE_TABLE_ID, new org.apache.thrift.meta_data.FieldMetaData("remoteTableId", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
-      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(getServicerAddress_args.class, metaDataMap);
-    }
-
-    public getServicerAddress_args() {
-    }
-
-    public getServicerAddress_args(
-      java.lang.String remoteTableId,
-      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials)
-    {
-      this();
-      this.remoteTableId = remoteTableId;
-      this.credentials = credentials;
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public getServicerAddress_args(getServicerAddress_args other) {
-      if (other.isSetRemoteTableId()) {
-        this.remoteTableId = other.remoteTableId;
-      }
-      if (other.isSetCredentials()) {
-        this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
-      }
-    }
-
-    @Override
-    public getServicerAddress_args deepCopy() {
-      return new getServicerAddress_args(this);
-    }
-
-    @Override
-    public void clear() {
-      this.remoteTableId = null;
-      this.credentials = null;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.lang.String getRemoteTableId() {
-      return this.remoteTableId;
-    }
-
-    public getServicerAddress_args setRemoteTableId(@org.apache.thrift.annotation.Nullable java.lang.String remoteTableId) {
-      this.remoteTableId = remoteTableId;
-      return this;
-    }
-
-    public void unsetRemoteTableId() {
-      this.remoteTableId = null;
-    }
-
-    /** Returns true if field remoteTableId is set (has been assigned a value) and false otherwise */
-    public boolean isSetRemoteTableId() {
-      return this.remoteTableId != null;
-    }
-
-    public void setRemoteTableIdIsSet(boolean value) {
-      if (!value) {
-        this.remoteTableId = null;
-      }
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.securityImpl.thrift.TCredentials getCredentials() {
-      return this.credentials;
-    }
-
-    public getServicerAddress_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;
-      }
-    }
-
-    @Override
-    public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
-      switch (field) {
-      case REMOTE_TABLE_ID:
-        if (value == null) {
-          unsetRemoteTableId();
-        } else {
-          setRemoteTableId((java.lang.String)value);
-        }
-        break;
-
-      case CREDENTIALS:
-        if (value == null) {
-          unsetCredentials();
-        } else {
-          setCredentials((org.apache.accumulo.core.securityImpl.thrift.TCredentials)value);
-        }
-        break;
-
-      }
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    @Override
-    public java.lang.Object getFieldValue(_Fields field) {
-      switch (field) {
-      case REMOTE_TABLE_ID:
-        return getRemoteTableId();
-
-      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 */
-    @Override
-    public boolean isSet(_Fields field) {
-      if (field == null) {
-        throw new java.lang.IllegalArgumentException();
-      }
-
-      switch (field) {
-      case REMOTE_TABLE_ID:
-        return isSetRemoteTableId();
-      case CREDENTIALS:
-        return isSetCredentials();
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(java.lang.Object that) {
-      if (that instanceof getServicerAddress_args)
-        return this.equals((getServicerAddress_args)that);
-      return false;
-    }
-
-    public boolean equals(getServicerAddress_args that) {
-      if (that == null)
-        return false;
-      if (this == that)
-        return true;
-
-      boolean this_present_remoteTableId = true && this.isSetRemoteTableId();
-      boolean that_present_remoteTableId = true && that.isSetRemoteTableId();
-      if (this_present_remoteTableId || that_present_remoteTableId) {
-        if (!(this_present_remoteTableId && that_present_remoteTableId))
-          return false;
-        if (!this.remoteTableId.equals(that.remoteTableId))
-          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 + ((isSetRemoteTableId()) ? 131071 : 524287);
-      if (isSetRemoteTableId())
-        hashCode = hashCode * 8191 + remoteTableId.hashCode();
-
-      hashCode = hashCode * 8191 + ((isSetCredentials()) ? 131071 : 524287);
-      if (isSetCredentials())
-        hashCode = hashCode * 8191 + credentials.hashCode();
-
-      return hashCode;
-    }
-
-    @Override
-    public int compareTo(getServicerAddress_args other) {
-      if (!getClass().equals(other.getClass())) {
-        return getClass().getName().compareTo(other.getClass().getName());
-      }
-
-      int lastComparison = 0;
-
-      lastComparison = java.lang.Boolean.compare(isSetRemoteTableId(), other.isSetRemoteTableId());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetRemoteTableId()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.remoteTableId, other.remoteTableId);
-        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
-    @Override
-    public _Fields fieldForId(int fieldId) {
-      return _Fields.findByThriftId(fieldId);
-    }
-
-    @Override
-    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
-      scheme(iprot).read(iprot, this);
-    }
-
-    @Override
-    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("getServicerAddress_args(");
-      boolean first = true;
-
-      sb.append("remoteTableId:");
-      if (this.remoteTableId == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.remoteTableId);
-      }
-      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 (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 getServicerAddress_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public getServicerAddress_argsStandardScheme getScheme() {
-        return new getServicerAddress_argsStandardScheme();
-      }
-    }
-
-    private static class getServicerAddress_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<getServicerAddress_args> {
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol iprot, getServicerAddress_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: // REMOTE_TABLE_ID
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
-                struct.remoteTableId = iprot.readString();
-                struct.setRemoteTableIdIsSet(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();
-      }
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol oprot, getServicerAddress_args struct) throws org.apache.thrift.TException {
-        struct.validate();
-
-        oprot.writeStructBegin(STRUCT_DESC);
-        if (struct.remoteTableId != null) {
-          oprot.writeFieldBegin(REMOTE_TABLE_ID_FIELD_DESC);
-          oprot.writeString(struct.remoteTableId);
-          oprot.writeFieldEnd();
-        }
-        if (struct.credentials != null) {
-          oprot.writeFieldBegin(CREDENTIALS_FIELD_DESC);
-          struct.credentials.write(oprot);
-          oprot.writeFieldEnd();
-        }
-        oprot.writeFieldStop();
-        oprot.writeStructEnd();
-      }
-
-    }
-
-    private static class getServicerAddress_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public getServicerAddress_argsTupleScheme getScheme() {
-        return new getServicerAddress_argsTupleScheme();
-      }
-    }
-
-    private static class getServicerAddress_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<getServicerAddress_args> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, getServicerAddress_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.isSetRemoteTableId()) {
-          optionals.set(0);
-        }
-        if (struct.isSetCredentials()) {
-          optionals.set(1);
-        }
-        oprot.writeBitSet(optionals, 2);
-        if (struct.isSetRemoteTableId()) {
-          oprot.writeString(struct.remoteTableId);
-        }
-        if (struct.isSetCredentials()) {
-          struct.credentials.write(oprot);
-        }
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, getServicerAddress_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.remoteTableId = iprot.readString();
-          struct.setRemoteTableIdIsSet(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();
-    }
-  }
-
-  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-  public static class getServicerAddress_result implements org.apache.thrift.TBase<getServicerAddress_result, getServicerAddress_result._Fields>, java.io.Serializable, Cloneable, Comparable<getServicerAddress_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getServicerAddress_result");
-
-    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRING, (short)0);
-    private static final org.apache.thrift.protocol.TField E_FIELD_DESC = new org.apache.thrift.protocol.TField("e", org.apache.thrift.protocol.TType.STRUCT, (short)1);
-
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getServicerAddress_resultStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getServicerAddress_resultTupleSchemeFactory();
-
-    public @org.apache.thrift.annotation.Nullable java.lang.String success; // required
-    public @org.apache.thrift.annotation.Nullable ReplicationCoordinatorException e; // 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"),
-      E((short)1, "e");
-
-      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: // E
-            return E;
-          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;
-      }
-
-      @Override
-      public short getThriftFieldId() {
-        return _thriftId;
-      }
-
-      @Override
-      public java.lang.String getFieldName() {
-        return _fieldName;
-      }
-    }
-
-    // isset id assignments
-    public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
-    static {
-      java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
-      tmpMap.put(_Fields.E, new org.apache.thrift.meta_data.FieldMetaData("e", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ReplicationCoordinatorException.class)));
-      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getServicerAddress_result.class, metaDataMap);
-    }
-
-    public getServicerAddress_result() {
-    }
-
-    public getServicerAddress_result(
-      java.lang.String success,
-      ReplicationCoordinatorException e)
-    {
-      this();
-      this.success = success;
-      this.e = e;
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public getServicerAddress_result(getServicerAddress_result other) {
-      if (other.isSetSuccess()) {
-        this.success = other.success;
-      }
-      if (other.isSetE()) {
-        this.e = new ReplicationCoordinatorException(other.e);
-      }
-    }
-
-    @Override
-    public getServicerAddress_result deepCopy() {
-      return new getServicerAddress_result(this);
-    }
-
-    @Override
-    public void clear() {
-      this.success = null;
-      this.e = null;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.lang.String getSuccess() {
-      return this.success;
-    }
-
-    public getServicerAddress_result setSuccess(@org.apache.thrift.annotation.Nullable java.lang.String success) {
-      this.success = success;
-      return this;
-    }
-
-    public void unsetSuccess() {
-      this.success = null;
-    }
-
-    /** Returns true if field success is set (has been assigned a value) and false otherwise */
-    public boolean isSetSuccess() {
-      return this.success != null;
-    }
-
-    public void setSuccessIsSet(boolean value) {
-      if (!value) {
-        this.success = null;
-      }
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public ReplicationCoordinatorException getE() {
-      return this.e;
-    }
-
-    public getServicerAddress_result setE(@org.apache.thrift.annotation.Nullable ReplicationCoordinatorException e) {
-      this.e = e;
-      return this;
-    }
-
-    public void unsetE() {
-      this.e = null;
-    }
-
-    /** Returns true if field e is set (has been assigned a value) and false otherwise */
-    public boolean isSetE() {
-      return this.e != null;
-    }
-
-    public void setEIsSet(boolean value) {
-      if (!value) {
-        this.e = null;
-      }
-    }
-
-    @Override
-    public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
-      switch (field) {
-      case SUCCESS:
-        if (value == null) {
-          unsetSuccess();
-        } else {
-          setSuccess((java.lang.String)value);
-        }
-        break;
-
-      case E:
-        if (value == null) {
-          unsetE();
-        } else {
-          setE((ReplicationCoordinatorException)value);
-        }
-        break;
-
-      }
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    @Override
-    public java.lang.Object getFieldValue(_Fields field) {
-      switch (field) {
-      case SUCCESS:
-        return getSuccess();
-
-      case E:
-        return getE();
-
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
-    @Override
-    public boolean isSet(_Fields field) {
-      if (field == null) {
-        throw new java.lang.IllegalArgumentException();
-      }
-
-      switch (field) {
-      case SUCCESS:
-        return isSetSuccess();
-      case E:
-        return isSetE();
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(java.lang.Object that) {
-      if (that instanceof getServicerAddress_result)
-        return this.equals((getServicerAddress_result)that);
-      return false;
-    }
-
-    public boolean equals(getServicerAddress_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_e = true && this.isSetE();
-      boolean that_present_e = true && that.isSetE();
-      if (this_present_e || that_present_e) {
-        if (!(this_present_e && that_present_e))
-          return false;
-        if (!this.e.equals(that.e))
-          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 + ((isSetE()) ? 131071 : 524287);
-      if (isSetE())
-        hashCode = hashCode * 8191 + e.hashCode();
-
-      return hashCode;
-    }
-
-    @Override
-    public int compareTo(getServicerAddress_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(isSetE(), other.isSetE());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetE()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.e, other.e);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      return 0;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    @Override
-    public _Fields fieldForId(int fieldId) {
-      return _Fields.findByThriftId(fieldId);
-    }
-
-    @Override
-    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("getServicerAddress_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("e:");
-      if (this.e == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.e);
-      }
-      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 getServicerAddress_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public getServicerAddress_resultStandardScheme getScheme() {
-        return new getServicerAddress_resultStandardScheme();
-      }
-    }
-
-    private static class getServicerAddress_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<getServicerAddress_result> {
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol iprot, getServicerAddress_result struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TField schemeField;
-        iprot.readStructBegin();
-        while (true)
-        {
-          schemeField = iprot.readFieldBegin();
-          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
-            break;
-          }
-          switch (schemeField.id) {
-            case 0: // SUCCESS
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
-                struct.success = iprot.readString();
-                struct.setSuccessIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 1: // E
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.e = new ReplicationCoordinatorException();
-                struct.e.read(iprot);
-                struct.setEIsSet(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();
-      }
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol oprot, getServicerAddress_result struct) throws org.apache.thrift.TException {
-        struct.validate();
-
-        oprot.writeStructBegin(STRUCT_DESC);
-        if (struct.success != null) {
-          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
-          oprot.writeString(struct.success);
-          oprot.writeFieldEnd();
-        }
-        if (struct.e != null) {
-          oprot.writeFieldBegin(E_FIELD_DESC);
-          struct.e.write(oprot);
-          oprot.writeFieldEnd();
-        }
-        oprot.writeFieldStop();
-        oprot.writeStructEnd();
-      }
-
-    }
-
-    private static class getServicerAddress_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public getServicerAddress_resultTupleScheme getScheme() {
-        return new getServicerAddress_resultTupleScheme();
-      }
-    }
-
-    private static class getServicerAddress_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<getServicerAddress_result> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, getServicerAddress_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.isSetE()) {
-          optionals.set(1);
-        }
-        oprot.writeBitSet(optionals, 2);
-        if (struct.isSetSuccess()) {
-          oprot.writeString(struct.success);
-        }
-        if (struct.isSetE()) {
-          struct.e.write(oprot);
-        }
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, getServicerAddress_result struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
-        java.util.BitSet incoming = iprot.readBitSet(2);
-        if (incoming.get(0)) {
-          struct.success = iprot.readString();
-          struct.setSuccessIsSet(true);
-        }
-        if (incoming.get(1)) {
-          struct.e = new ReplicationCoordinatorException();
-          struct.e.read(iprot);
-          struct.setEIsSet(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();
-    }
-  }
-
-  private static void unusedMethod() {}
-}
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/replication/thrift/ReplicationCoordinatorErrorCode.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/replication/thrift/ReplicationCoordinatorErrorCode.java
deleted file mode 100644
index 2f1c337..0000000
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/replication/thrift/ReplicationCoordinatorErrorCode.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/**
- * Autogenerated by Thrift Compiler (0.17.0)
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- *  @generated
- */
-package org.apache.accumulo.core.replication.thrift;
-
-
-public enum ReplicationCoordinatorErrorCode implements org.apache.thrift.TEnum {
-  NO_AVAILABLE_SERVERS(0),
-  SERVICE_CONFIGURATION_UNAVAILABLE(1),
-  CANNOT_AUTHENTICATE(2);
-
-  private final int value;
-
-  private ReplicationCoordinatorErrorCode(int value) {
-    this.value = value;
-  }
-
-  /**
-   * Get the integer value of this enum value, as defined in the Thrift IDL.
-   */
-  @Override
-  public int getValue() {
-    return value;
-  }
-
-  /**
-   * Find a the enum type by its integer value, as defined in the Thrift IDL.
-   * @return null if the value is not found.
-   */
-  @org.apache.thrift.annotation.Nullable
-  public static ReplicationCoordinatorErrorCode findByValue(int value) { 
-    switch (value) {
-      case 0:
-        return NO_AVAILABLE_SERVERS;
-      case 1:
-        return SERVICE_CONFIGURATION_UNAVAILABLE;
-      case 2:
-        return CANNOT_AUTHENTICATE;
-      default:
-        return null;
-    }
-  }
-}
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/replication/thrift/ReplicationCoordinatorException.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/replication/thrift/ReplicationCoordinatorException.java
deleted file mode 100644
index 48ad99a..0000000
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/replication/thrift/ReplicationCoordinatorException.java
+++ /dev/null
@@ -1,527 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/**
- * Autogenerated by Thrift Compiler (0.17.0)
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- *  @generated
- */
-package org.apache.accumulo.core.replication.thrift;
-
-@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-public class ReplicationCoordinatorException extends org.apache.thrift.TException implements org.apache.thrift.TBase<ReplicationCoordinatorException, ReplicationCoordinatorException._Fields>, java.io.Serializable, Cloneable, Comparable<ReplicationCoordinatorException> {
-  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ReplicationCoordinatorException");
-
-  private static final org.apache.thrift.protocol.TField CODE_FIELD_DESC = new org.apache.thrift.protocol.TField("code", org.apache.thrift.protocol.TType.I32, (short)1);
-  private static final org.apache.thrift.protocol.TField REASON_FIELD_DESC = new org.apache.thrift.protocol.TField("reason", org.apache.thrift.protocol.TType.STRING, (short)2);
-
-  private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new ReplicationCoordinatorExceptionStandardSchemeFactory();
-  private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new ReplicationCoordinatorExceptionTupleSchemeFactory();
-
-  /**
-   * 
-   * @see ReplicationCoordinatorErrorCode
-   */
-  public @org.apache.thrift.annotation.Nullable ReplicationCoordinatorErrorCode code; // required
-  public @org.apache.thrift.annotation.Nullable java.lang.String reason; // 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 {
-    /**
-     * 
-     * @see ReplicationCoordinatorErrorCode
-     */
-    CODE((short)1, "code"),
-    REASON((short)2, "reason");
-
-    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: // CODE
-          return CODE;
-        case 2: // REASON
-          return REASON;
-        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;
-    }
-
-    @Override
-    public short getThriftFieldId() {
-      return _thriftId;
-    }
-
-    @Override
-    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.CODE, new org.apache.thrift.meta_data.FieldMetaData("code", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-        new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, ReplicationCoordinatorErrorCode.class)));
-    tmpMap.put(_Fields.REASON, new org.apache.thrift.meta_data.FieldMetaData("reason", 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(ReplicationCoordinatorException.class, metaDataMap);
-  }
-
-  public ReplicationCoordinatorException() {
-  }
-
-  public ReplicationCoordinatorException(
-    ReplicationCoordinatorErrorCode code,
-    java.lang.String reason)
-  {
-    this();
-    this.code = code;
-    this.reason = reason;
-  }
-
-  /**
-   * Performs a deep copy on <i>other</i>.
-   */
-  public ReplicationCoordinatorException(ReplicationCoordinatorException other) {
-    if (other.isSetCode()) {
-      this.code = other.code;
-    }
-    if (other.isSetReason()) {
-      this.reason = other.reason;
-    }
-  }
-
-  @Override
-  public ReplicationCoordinatorException deepCopy() {
-    return new ReplicationCoordinatorException(this);
-  }
-
-  @Override
-  public void clear() {
-    this.code = null;
-    this.reason = null;
-  }
-
-  /**
-   * 
-   * @see ReplicationCoordinatorErrorCode
-   */
-  @org.apache.thrift.annotation.Nullable
-  public ReplicationCoordinatorErrorCode getCode() {
-    return this.code;
-  }
-
-  /**
-   * 
-   * @see ReplicationCoordinatorErrorCode
-   */
-  public ReplicationCoordinatorException setCode(@org.apache.thrift.annotation.Nullable ReplicationCoordinatorErrorCode code) {
-    this.code = code;
-    return this;
-  }
-
-  public void unsetCode() {
-    this.code = null;
-  }
-
-  /** Returns true if field code is set (has been assigned a value) and false otherwise */
-  public boolean isSetCode() {
-    return this.code != null;
-  }
-
-  public void setCodeIsSet(boolean value) {
-    if (!value) {
-      this.code = null;
-    }
-  }
-
-  @org.apache.thrift.annotation.Nullable
-  public java.lang.String getReason() {
-    return this.reason;
-  }
-
-  public ReplicationCoordinatorException setReason(@org.apache.thrift.annotation.Nullable java.lang.String reason) {
-    this.reason = reason;
-    return this;
-  }
-
-  public void unsetReason() {
-    this.reason = null;
-  }
-
-  /** Returns true if field reason is set (has been assigned a value) and false otherwise */
-  public boolean isSetReason() {
-    return this.reason != null;
-  }
-
-  public void setReasonIsSet(boolean value) {
-    if (!value) {
-      this.reason = null;
-    }
-  }
-
-  @Override
-  public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
-    switch (field) {
-    case CODE:
-      if (value == null) {
-        unsetCode();
-      } else {
-        setCode((ReplicationCoordinatorErrorCode)value);
-      }
-      break;
-
-    case REASON:
-      if (value == null) {
-        unsetReason();
-      } else {
-        setReason((java.lang.String)value);
-      }
-      break;
-
-    }
-  }
-
-  @org.apache.thrift.annotation.Nullable
-  @Override
-  public java.lang.Object getFieldValue(_Fields field) {
-    switch (field) {
-    case CODE:
-      return getCode();
-
-    case REASON:
-      return getReason();
-
-    }
-    throw new java.lang.IllegalStateException();
-  }
-
-  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
-  @Override
-  public boolean isSet(_Fields field) {
-    if (field == null) {
-      throw new java.lang.IllegalArgumentException();
-    }
-
-    switch (field) {
-    case CODE:
-      return isSetCode();
-    case REASON:
-      return isSetReason();
-    }
-    throw new java.lang.IllegalStateException();
-  }
-
-  @Override
-  public boolean equals(java.lang.Object that) {
-    if (that instanceof ReplicationCoordinatorException)
-      return this.equals((ReplicationCoordinatorException)that);
-    return false;
-  }
-
-  public boolean equals(ReplicationCoordinatorException that) {
-    if (that == null)
-      return false;
-    if (this == that)
-      return true;
-
-    boolean this_present_code = true && this.isSetCode();
-    boolean that_present_code = true && that.isSetCode();
-    if (this_present_code || that_present_code) {
-      if (!(this_present_code && that_present_code))
-        return false;
-      if (!this.code.equals(that.code))
-        return false;
-    }
-
-    boolean this_present_reason = true && this.isSetReason();
-    boolean that_present_reason = true && that.isSetReason();
-    if (this_present_reason || that_present_reason) {
-      if (!(this_present_reason && that_present_reason))
-        return false;
-      if (!this.reason.equals(that.reason))
-        return false;
-    }
-
-    return true;
-  }
-
-  @Override
-  public int hashCode() {
-    int hashCode = 1;
-
-    hashCode = hashCode * 8191 + ((isSetCode()) ? 131071 : 524287);
-    if (isSetCode())
-      hashCode = hashCode * 8191 + code.getValue();
-
-    hashCode = hashCode * 8191 + ((isSetReason()) ? 131071 : 524287);
-    if (isSetReason())
-      hashCode = hashCode * 8191 + reason.hashCode();
-
-    return hashCode;
-  }
-
-  @Override
-  public int compareTo(ReplicationCoordinatorException other) {
-    if (!getClass().equals(other.getClass())) {
-      return getClass().getName().compareTo(other.getClass().getName());
-    }
-
-    int lastComparison = 0;
-
-    lastComparison = java.lang.Boolean.compare(isSetCode(), other.isSetCode());
-    if (lastComparison != 0) {
-      return lastComparison;
-    }
-    if (isSetCode()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.code, other.code);
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-    }
-    lastComparison = java.lang.Boolean.compare(isSetReason(), other.isSetReason());
-    if (lastComparison != 0) {
-      return lastComparison;
-    }
-    if (isSetReason()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.reason, other.reason);
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-    }
-    return 0;
-  }
-
-  @org.apache.thrift.annotation.Nullable
-  @Override
-  public _Fields fieldForId(int fieldId) {
-    return _Fields.findByThriftId(fieldId);
-  }
-
-  @Override
-  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
-    scheme(iprot).read(iprot, this);
-  }
-
-  @Override
-  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("ReplicationCoordinatorException(");
-    boolean first = true;
-
-    sb.append("code:");
-    if (this.code == null) {
-      sb.append("null");
-    } else {
-      sb.append(this.code);
-    }
-    first = false;
-    if (!first) sb.append(", ");
-    sb.append("reason:");
-    if (this.reason == null) {
-      sb.append("null");
-    } else {
-      sb.append(this.reason);
-    }
-    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 ReplicationCoordinatorExceptionStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-    @Override
-    public ReplicationCoordinatorExceptionStandardScheme getScheme() {
-      return new ReplicationCoordinatorExceptionStandardScheme();
-    }
-  }
-
-  private static class ReplicationCoordinatorExceptionStandardScheme extends org.apache.thrift.scheme.StandardScheme<ReplicationCoordinatorException> {
-
-    @Override
-    public void read(org.apache.thrift.protocol.TProtocol iprot, ReplicationCoordinatorException 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: // CODE
-            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
-              struct.code = org.apache.accumulo.core.replication.thrift.ReplicationCoordinatorErrorCode.findByValue(iprot.readI32());
-              struct.setCodeIsSet(true);
-            } else { 
-              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-            }
-            break;
-          case 2: // REASON
-            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
-              struct.reason = iprot.readString();
-              struct.setReasonIsSet(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();
-    }
-
-    @Override
-    public void write(org.apache.thrift.protocol.TProtocol oprot, ReplicationCoordinatorException struct) throws org.apache.thrift.TException {
-      struct.validate();
-
-      oprot.writeStructBegin(STRUCT_DESC);
-      if (struct.code != null) {
-        oprot.writeFieldBegin(CODE_FIELD_DESC);
-        oprot.writeI32(struct.code.getValue());
-        oprot.writeFieldEnd();
-      }
-      if (struct.reason != null) {
-        oprot.writeFieldBegin(REASON_FIELD_DESC);
-        oprot.writeString(struct.reason);
-        oprot.writeFieldEnd();
-      }
-      oprot.writeFieldStop();
-      oprot.writeStructEnd();
-    }
-
-  }
-
-  private static class ReplicationCoordinatorExceptionTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-    @Override
-    public ReplicationCoordinatorExceptionTupleScheme getScheme() {
-      return new ReplicationCoordinatorExceptionTupleScheme();
-    }
-  }
-
-  private static class ReplicationCoordinatorExceptionTupleScheme extends org.apache.thrift.scheme.TupleScheme<ReplicationCoordinatorException> {
-
-    @Override
-    public void write(org.apache.thrift.protocol.TProtocol prot, ReplicationCoordinatorException 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.isSetCode()) {
-        optionals.set(0);
-      }
-      if (struct.isSetReason()) {
-        optionals.set(1);
-      }
-      oprot.writeBitSet(optionals, 2);
-      if (struct.isSetCode()) {
-        oprot.writeI32(struct.code.getValue());
-      }
-      if (struct.isSetReason()) {
-        oprot.writeString(struct.reason);
-      }
-    }
-
-    @Override
-    public void read(org.apache.thrift.protocol.TProtocol prot, ReplicationCoordinatorException 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.code = org.apache.accumulo.core.replication.thrift.ReplicationCoordinatorErrorCode.findByValue(iprot.readI32());
-        struct.setCodeIsSet(true);
-      }
-      if (incoming.get(1)) {
-        struct.reason = iprot.readString();
-        struct.setReasonIsSet(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();
-  }
-  private static void unusedMethod() {}
-}
-
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/replication/thrift/ReplicationServicer.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/replication/thrift/ReplicationServicer.java
deleted file mode 100644
index 7d7e41c..0000000
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/replication/thrift/ReplicationServicer.java
+++ /dev/null
@@ -1,2638 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/**
- * Autogenerated by Thrift Compiler (0.17.0)
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- *  @generated
- */
-package org.apache.accumulo.core.replication.thrift;
-
-@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-public class ReplicationServicer {
-
-  public interface Iface {
-
-    public long replicateLog(java.lang.String remoteTableId, WalEdits data, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws RemoteReplicationException, org.apache.thrift.TException;
-
-    public long replicateKeyValues(java.lang.String remoteTableId, KeyValues data, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws RemoteReplicationException, org.apache.thrift.TException;
-
-  }
-
-  public interface AsyncIface {
-
-    public void replicateLog(java.lang.String remoteTableId, WalEdits data, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler) throws org.apache.thrift.TException;
-
-    public void replicateKeyValues(java.lang.String remoteTableId, KeyValues data, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler) throws org.apache.thrift.TException;
-
-  }
-
-  public static class Client extends org.apache.thrift.TServiceClient implements Iface {
-    public static class Factory implements org.apache.thrift.TServiceClientFactory<Client> {
-      public Factory() {}
-      @Override
-      public Client getClient(org.apache.thrift.protocol.TProtocol prot) {
-        return new Client(prot);
-      }
-      @Override
-      public Client getClient(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TProtocol oprot) {
-        return new Client(iprot, oprot);
-      }
-    }
-
-    public Client(org.apache.thrift.protocol.TProtocol prot)
-    {
-      super(prot, prot);
-    }
-
-    public Client(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TProtocol oprot) {
-      super(iprot, oprot);
-    }
-
-    @Override
-    public long replicateLog(java.lang.String remoteTableId, WalEdits data, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws RemoteReplicationException, org.apache.thrift.TException
-    {
-      send_replicateLog(remoteTableId, data, credentials);
-      return recv_replicateLog();
-    }
-
-    public void send_replicateLog(java.lang.String remoteTableId, WalEdits data, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
-    {
-      replicateLog_args args = new replicateLog_args();
-      args.setRemoteTableId(remoteTableId);
-      args.setData(data);
-      args.setCredentials(credentials);
-      sendBase("replicateLog", args);
-    }
-
-    public long recv_replicateLog() throws RemoteReplicationException, org.apache.thrift.TException
-    {
-      replicateLog_result result = new replicateLog_result();
-      receiveBase(result, "replicateLog");
-      if (result.isSetSuccess()) {
-        return result.success;
-      }
-      if (result.e != null) {
-        throw result.e;
-      }
-      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "replicateLog failed: unknown result");
-    }
-
-    @Override
-    public long replicateKeyValues(java.lang.String remoteTableId, KeyValues data, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws RemoteReplicationException, org.apache.thrift.TException
-    {
-      send_replicateKeyValues(remoteTableId, data, credentials);
-      return recv_replicateKeyValues();
-    }
-
-    public void send_replicateKeyValues(java.lang.String remoteTableId, KeyValues data, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
-    {
-      replicateKeyValues_args args = new replicateKeyValues_args();
-      args.setRemoteTableId(remoteTableId);
-      args.setData(data);
-      args.setCredentials(credentials);
-      sendBase("replicateKeyValues", args);
-    }
-
-    public long recv_replicateKeyValues() throws RemoteReplicationException, org.apache.thrift.TException
-    {
-      replicateKeyValues_result result = new replicateKeyValues_result();
-      receiveBase(result, "replicateKeyValues");
-      if (result.isSetSuccess()) {
-        return result.success;
-      }
-      if (result.e != null) {
-        throw result.e;
-      }
-      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "replicateKeyValues failed: unknown result");
-    }
-
-  }
-  public static class AsyncClient extends org.apache.thrift.async.TAsyncClient implements AsyncIface {
-    public static class Factory implements org.apache.thrift.async.TAsyncClientFactory<AsyncClient> {
-      private org.apache.thrift.async.TAsyncClientManager clientManager;
-      private org.apache.thrift.protocol.TProtocolFactory protocolFactory;
-      public Factory(org.apache.thrift.async.TAsyncClientManager clientManager, org.apache.thrift.protocol.TProtocolFactory protocolFactory) {
-        this.clientManager = clientManager;
-        this.protocolFactory = protocolFactory;
-      }
-    @Override
-      public AsyncClient getAsyncClient(org.apache.thrift.transport.TNonblockingTransport transport) {
-        return new AsyncClient(protocolFactory, clientManager, transport);
-      }
-    }
-
-    public AsyncClient(org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.async.TAsyncClientManager clientManager, org.apache.thrift.transport.TNonblockingTransport transport) {
-      super(protocolFactory, clientManager, transport);
-    }
-
-    @Override
-    public void replicateLog(java.lang.String remoteTableId, WalEdits data, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler) throws org.apache.thrift.TException {
-      checkReady();
-      replicateLog_call method_call = new replicateLog_call(remoteTableId, data, credentials, resultHandler, this, ___protocolFactory, ___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
-    }
-
-    public static class replicateLog_call extends org.apache.thrift.async.TAsyncMethodCall<java.lang.Long> {
-      private java.lang.String remoteTableId;
-      private WalEdits data;
-      private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
-      public replicateLog_call(java.lang.String remoteTableId, WalEdits data, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> 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.remoteTableId = remoteTableId;
-        this.data = data;
-        this.credentials = credentials;
-      }
-
-      @Override
-      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("replicateLog", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        replicateLog_args args = new replicateLog_args();
-        args.setRemoteTableId(remoteTableId);
-        args.setData(data);
-        args.setCredentials(credentials);
-        args.write(prot);
-        prot.writeMessageEnd();
-      }
-
-      @Override
-      public java.lang.Long getResult() throws RemoteReplicationException, 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_replicateLog();
-      }
-    }
-
-    @Override
-    public void replicateKeyValues(java.lang.String remoteTableId, KeyValues data, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler) throws org.apache.thrift.TException {
-      checkReady();
-      replicateKeyValues_call method_call = new replicateKeyValues_call(remoteTableId, data, credentials, resultHandler, this, ___protocolFactory, ___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
-    }
-
-    public static class replicateKeyValues_call extends org.apache.thrift.async.TAsyncMethodCall<java.lang.Long> {
-      private java.lang.String remoteTableId;
-      private KeyValues data;
-      private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
-      public replicateKeyValues_call(java.lang.String remoteTableId, KeyValues data, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> 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.remoteTableId = remoteTableId;
-        this.data = data;
-        this.credentials = credentials;
-      }
-
-      @Override
-      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("replicateKeyValues", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        replicateKeyValues_args args = new replicateKeyValues_args();
-        args.setRemoteTableId(remoteTableId);
-        args.setData(data);
-        args.setCredentials(credentials);
-        args.write(prot);
-        prot.writeMessageEnd();
-      }
-
-      @Override
-      public java.lang.Long getResult() throws RemoteReplicationException, 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_replicateKeyValues();
-      }
-    }
-
-  }
-
-  public static class Processor<I extends Iface> extends org.apache.thrift.TBaseProcessor<I> implements org.apache.thrift.TProcessor {
-    private static final org.slf4j.Logger _LOGGER = org.slf4j.LoggerFactory.getLogger(Processor.class.getName());
-    public Processor(I iface) {
-      super(iface, getProcessMap(new java.util.HashMap<java.lang.String, org.apache.thrift.ProcessFunction<I, ? extends org.apache.thrift.TBase>>()));
-    }
-
-    protected Processor(I iface, java.util.Map<java.lang.String, org.apache.thrift.ProcessFunction<I, ? extends org.apache.thrift.TBase>> processMap) {
-      super(iface, getProcessMap(processMap));
-    }
-
-    private static <I extends Iface> java.util.Map<java.lang.String,  org.apache.thrift.ProcessFunction<I, ? extends org.apache.thrift.TBase>> getProcessMap(java.util.Map<java.lang.String, org.apache.thrift.ProcessFunction<I, ? extends  org.apache.thrift.TBase>> processMap) {
-      processMap.put("replicateLog", new replicateLog());
-      processMap.put("replicateKeyValues", new replicateKeyValues());
-      return processMap;
-    }
-
-    public static class replicateLog<I extends Iface> extends org.apache.thrift.ProcessFunction<I, replicateLog_args> {
-      public replicateLog() {
-        super("replicateLog");
-      }
-
-      @Override
-      public replicateLog_args getEmptyArgsInstance() {
-        return new replicateLog_args();
-      }
-
-      @Override
-      protected boolean isOneway() {
-        return false;
-      }
-
-      @Override
-      protected boolean rethrowUnhandledExceptions() {
-        return false;
-      }
-
-      @Override
-      public replicateLog_result getResult(I iface, replicateLog_args args) throws org.apache.thrift.TException {
-        replicateLog_result result = new replicateLog_result();
-        try {
-          result.success = iface.replicateLog(args.remoteTableId, args.data, args.credentials);
-          result.setSuccessIsSet(true);
-        } catch (RemoteReplicationException e) {
-          result.e = e;
-        }
-        return result;
-      }
-    }
-
-    public static class replicateKeyValues<I extends Iface> extends org.apache.thrift.ProcessFunction<I, replicateKeyValues_args> {
-      public replicateKeyValues() {
-        super("replicateKeyValues");
-      }
-
-      @Override
-      public replicateKeyValues_args getEmptyArgsInstance() {
-        return new replicateKeyValues_args();
-      }
-
-      @Override
-      protected boolean isOneway() {
-        return false;
-      }
-
-      @Override
-      protected boolean rethrowUnhandledExceptions() {
-        return false;
-      }
-
-      @Override
-      public replicateKeyValues_result getResult(I iface, replicateKeyValues_args args) throws org.apache.thrift.TException {
-        replicateKeyValues_result result = new replicateKeyValues_result();
-        try {
-          result.success = iface.replicateKeyValues(args.remoteTableId, args.data, args.credentials);
-          result.setSuccessIsSet(true);
-        } catch (RemoteReplicationException e) {
-          result.e = e;
-        }
-        return result;
-      }
-    }
-
-  }
-
-  public static class AsyncProcessor<I extends AsyncIface> extends org.apache.thrift.TBaseAsyncProcessor<I> {
-    private static final org.slf4j.Logger _LOGGER = org.slf4j.LoggerFactory.getLogger(AsyncProcessor.class.getName());
-    public AsyncProcessor(I iface) {
-      super(iface, getProcessMap(new java.util.HashMap<java.lang.String, org.apache.thrift.AsyncProcessFunction<I, ? extends org.apache.thrift.TBase, ?>>()));
-    }
-
-    protected AsyncProcessor(I iface, java.util.Map<java.lang.String,  org.apache.thrift.AsyncProcessFunction<I, ? extends  org.apache.thrift.TBase, ?>> processMap) {
-      super(iface, getProcessMap(processMap));
-    }
-
-    private static <I extends AsyncIface> java.util.Map<java.lang.String,  org.apache.thrift.AsyncProcessFunction<I, ? extends  org.apache.thrift.TBase,?>> getProcessMap(java.util.Map<java.lang.String,  org.apache.thrift.AsyncProcessFunction<I, ? extends  org.apache.thrift.TBase, ?>> processMap) {
-      processMap.put("replicateLog", new replicateLog());
-      processMap.put("replicateKeyValues", new replicateKeyValues());
-      return processMap;
-    }
-
-    public static class replicateLog<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, replicateLog_args, java.lang.Long> {
-      public replicateLog() {
-        super("replicateLog");
-      }
-
-      @Override
-      public replicateLog_args getEmptyArgsInstance() {
-        return new replicateLog_args();
-      }
-
-      @Override
-      public org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> 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.Long>() { 
-          @Override
-          public void onComplete(java.lang.Long o) {
-            replicateLog_result result = new replicateLog_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);
-            }
-          }
-          @Override
-          public void onError(java.lang.Exception e) {
-            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
-            org.apache.thrift.TSerializable msg;
-            replicateLog_result result = new replicateLog_result();
-            if (e instanceof RemoteReplicationException) {
-              result.e = (RemoteReplicationException) e;
-              result.setEIsSet(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();
-            }
-          }
-        };
-      }
-
-      @Override
-      protected boolean isOneway() {
-        return false;
-      }
-
-      @Override
-      public void start(I iface, replicateLog_args args, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler) throws org.apache.thrift.TException {
-        iface.replicateLog(args.remoteTableId, args.data, args.credentials,resultHandler);
-      }
-    }
-
-    public static class replicateKeyValues<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, replicateKeyValues_args, java.lang.Long> {
-      public replicateKeyValues() {
-        super("replicateKeyValues");
-      }
-
-      @Override
-      public replicateKeyValues_args getEmptyArgsInstance() {
-        return new replicateKeyValues_args();
-      }
-
-      @Override
-      public org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> 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.Long>() { 
-          @Override
-          public void onComplete(java.lang.Long o) {
-            replicateKeyValues_result result = new replicateKeyValues_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);
-            }
-          }
-          @Override
-          public void onError(java.lang.Exception e) {
-            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
-            org.apache.thrift.TSerializable msg;
-            replicateKeyValues_result result = new replicateKeyValues_result();
-            if (e instanceof RemoteReplicationException) {
-              result.e = (RemoteReplicationException) e;
-              result.setEIsSet(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();
-            }
-          }
-        };
-      }
-
-      @Override
-      protected boolean isOneway() {
-        return false;
-      }
-
-      @Override
-      public void start(I iface, replicateKeyValues_args args, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler) throws org.apache.thrift.TException {
-        iface.replicateKeyValues(args.remoteTableId, args.data, args.credentials,resultHandler);
-      }
-    }
-
-  }
-
-  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-  public static class replicateLog_args implements org.apache.thrift.TBase<replicateLog_args, replicateLog_args._Fields>, java.io.Serializable, Cloneable, Comparable<replicateLog_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("replicateLog_args");
-
-    private static final org.apache.thrift.protocol.TField REMOTE_TABLE_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("remoteTableId", org.apache.thrift.protocol.TType.STRING, (short)1);
-    private static final org.apache.thrift.protocol.TField DATA_FIELD_DESC = new org.apache.thrift.protocol.TField("data", 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.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new replicateLog_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new replicateLog_argsTupleSchemeFactory();
-
-    public @org.apache.thrift.annotation.Nullable java.lang.String remoteTableId; // required
-    public @org.apache.thrift.annotation.Nullable WalEdits data; // 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 {
-      REMOTE_TABLE_ID((short)1, "remoteTableId"),
-      DATA((short)2, "data"),
-      CREDENTIALS((short)3, "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: // REMOTE_TABLE_ID
-            return REMOTE_TABLE_ID;
-          case 2: // DATA
-            return DATA;
-          case 3: // 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;
-      }
-
-      @Override
-      public short getThriftFieldId() {
-        return _thriftId;
-      }
-
-      @Override
-      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.REMOTE_TABLE_ID, new org.apache.thrift.meta_data.FieldMetaData("remoteTableId", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
-      tmpMap.put(_Fields.DATA, new org.apache.thrift.meta_data.FieldMetaData("data", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, WalEdits.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(replicateLog_args.class, metaDataMap);
-    }
-
-    public replicateLog_args() {
-    }
-
-    public replicateLog_args(
-      java.lang.String remoteTableId,
-      WalEdits data,
-      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials)
-    {
-      this();
-      this.remoteTableId = remoteTableId;
-      this.data = data;
-      this.credentials = credentials;
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public replicateLog_args(replicateLog_args other) {
-      if (other.isSetRemoteTableId()) {
-        this.remoteTableId = other.remoteTableId;
-      }
-      if (other.isSetData()) {
-        this.data = new WalEdits(other.data);
-      }
-      if (other.isSetCredentials()) {
-        this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
-      }
-    }
-
-    @Override
-    public replicateLog_args deepCopy() {
-      return new replicateLog_args(this);
-    }
-
-    @Override
-    public void clear() {
-      this.remoteTableId = null;
-      this.data = null;
-      this.credentials = null;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.lang.String getRemoteTableId() {
-      return this.remoteTableId;
-    }
-
-    public replicateLog_args setRemoteTableId(@org.apache.thrift.annotation.Nullable java.lang.String remoteTableId) {
-      this.remoteTableId = remoteTableId;
-      return this;
-    }
-
-    public void unsetRemoteTableId() {
-      this.remoteTableId = null;
-    }
-
-    /** Returns true if field remoteTableId is set (has been assigned a value) and false otherwise */
-    public boolean isSetRemoteTableId() {
-      return this.remoteTableId != null;
-    }
-
-    public void setRemoteTableIdIsSet(boolean value) {
-      if (!value) {
-        this.remoteTableId = null;
-      }
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public WalEdits getData() {
-      return this.data;
-    }
-
-    public replicateLog_args setData(@org.apache.thrift.annotation.Nullable WalEdits data) {
-      this.data = data;
-      return this;
-    }
-
-    public void unsetData() {
-      this.data = null;
-    }
-
-    /** Returns true if field data is set (has been assigned a value) and false otherwise */
-    public boolean isSetData() {
-      return this.data != null;
-    }
-
-    public void setDataIsSet(boolean value) {
-      if (!value) {
-        this.data = null;
-      }
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.securityImpl.thrift.TCredentials getCredentials() {
-      return this.credentials;
-    }
-
-    public replicateLog_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;
-      }
-    }
-
-    @Override
-    public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
-      switch (field) {
-      case REMOTE_TABLE_ID:
-        if (value == null) {
-          unsetRemoteTableId();
-        } else {
-          setRemoteTableId((java.lang.String)value);
-        }
-        break;
-
-      case DATA:
-        if (value == null) {
-          unsetData();
-        } else {
-          setData((WalEdits)value);
-        }
-        break;
-
-      case CREDENTIALS:
-        if (value == null) {
-          unsetCredentials();
-        } else {
-          setCredentials((org.apache.accumulo.core.securityImpl.thrift.TCredentials)value);
-        }
-        break;
-
-      }
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    @Override
-    public java.lang.Object getFieldValue(_Fields field) {
-      switch (field) {
-      case REMOTE_TABLE_ID:
-        return getRemoteTableId();
-
-      case DATA:
-        return getData();
-
-      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 */
-    @Override
-    public boolean isSet(_Fields field) {
-      if (field == null) {
-        throw new java.lang.IllegalArgumentException();
-      }
-
-      switch (field) {
-      case REMOTE_TABLE_ID:
-        return isSetRemoteTableId();
-      case DATA:
-        return isSetData();
-      case CREDENTIALS:
-        return isSetCredentials();
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(java.lang.Object that) {
-      if (that instanceof replicateLog_args)
-        return this.equals((replicateLog_args)that);
-      return false;
-    }
-
-    public boolean equals(replicateLog_args that) {
-      if (that == null)
-        return false;
-      if (this == that)
-        return true;
-
-      boolean this_present_remoteTableId = true && this.isSetRemoteTableId();
-      boolean that_present_remoteTableId = true && that.isSetRemoteTableId();
-      if (this_present_remoteTableId || that_present_remoteTableId) {
-        if (!(this_present_remoteTableId && that_present_remoteTableId))
-          return false;
-        if (!this.remoteTableId.equals(that.remoteTableId))
-          return false;
-      }
-
-      boolean this_present_data = true && this.isSetData();
-      boolean that_present_data = true && that.isSetData();
-      if (this_present_data || that_present_data) {
-        if (!(this_present_data && that_present_data))
-          return false;
-        if (!this.data.equals(that.data))
-          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 + ((isSetRemoteTableId()) ? 131071 : 524287);
-      if (isSetRemoteTableId())
-        hashCode = hashCode * 8191 + remoteTableId.hashCode();
-
-      hashCode = hashCode * 8191 + ((isSetData()) ? 131071 : 524287);
-      if (isSetData())
-        hashCode = hashCode * 8191 + data.hashCode();
-
-      hashCode = hashCode * 8191 + ((isSetCredentials()) ? 131071 : 524287);
-      if (isSetCredentials())
-        hashCode = hashCode * 8191 + credentials.hashCode();
-
-      return hashCode;
-    }
-
-    @Override
-    public int compareTo(replicateLog_args other) {
-      if (!getClass().equals(other.getClass())) {
-        return getClass().getName().compareTo(other.getClass().getName());
-      }
-
-      int lastComparison = 0;
-
-      lastComparison = java.lang.Boolean.compare(isSetRemoteTableId(), other.isSetRemoteTableId());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetRemoteTableId()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.remoteTableId, other.remoteTableId);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = java.lang.Boolean.compare(isSetData(), other.isSetData());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetData()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.data, other.data);
-        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
-    @Override
-    public _Fields fieldForId(int fieldId) {
-      return _Fields.findByThriftId(fieldId);
-    }
-
-    @Override
-    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
-      scheme(iprot).read(iprot, this);
-    }
-
-    @Override
-    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("replicateLog_args(");
-      boolean first = true;
-
-      sb.append("remoteTableId:");
-      if (this.remoteTableId == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.remoteTableId);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("data:");
-      if (this.data == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.data);
-      }
-      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 (data != null) {
-        data.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 replicateLog_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public replicateLog_argsStandardScheme getScheme() {
-        return new replicateLog_argsStandardScheme();
-      }
-    }
-
-    private static class replicateLog_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<replicateLog_args> {
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol iprot, replicateLog_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: // REMOTE_TABLE_ID
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
-                struct.remoteTableId = iprot.readString();
-                struct.setRemoteTableIdIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 2: // DATA
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.data = new WalEdits();
-                struct.data.read(iprot);
-                struct.setDataIsSet(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;
-            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();
-      }
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol oprot, replicateLog_args struct) throws org.apache.thrift.TException {
-        struct.validate();
-
-        oprot.writeStructBegin(STRUCT_DESC);
-        if (struct.remoteTableId != null) {
-          oprot.writeFieldBegin(REMOTE_TABLE_ID_FIELD_DESC);
-          oprot.writeString(struct.remoteTableId);
-          oprot.writeFieldEnd();
-        }
-        if (struct.data != null) {
-          oprot.writeFieldBegin(DATA_FIELD_DESC);
-          struct.data.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 replicateLog_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public replicateLog_argsTupleScheme getScheme() {
-        return new replicateLog_argsTupleScheme();
-      }
-    }
-
-    private static class replicateLog_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<replicateLog_args> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, replicateLog_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.isSetRemoteTableId()) {
-          optionals.set(0);
-        }
-        if (struct.isSetData()) {
-          optionals.set(1);
-        }
-        if (struct.isSetCredentials()) {
-          optionals.set(2);
-        }
-        oprot.writeBitSet(optionals, 3);
-        if (struct.isSetRemoteTableId()) {
-          oprot.writeString(struct.remoteTableId);
-        }
-        if (struct.isSetData()) {
-          struct.data.write(oprot);
-        }
-        if (struct.isSetCredentials()) {
-          struct.credentials.write(oprot);
-        }
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, replicateLog_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.remoteTableId = iprot.readString();
-          struct.setRemoteTableIdIsSet(true);
-        }
-        if (incoming.get(1)) {
-          struct.data = new WalEdits();
-          struct.data.read(iprot);
-          struct.setDataIsSet(true);
-        }
-        if (incoming.get(2)) {
-          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();
-    }
-  }
-
-  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-  public static class replicateLog_result implements org.apache.thrift.TBase<replicateLog_result, replicateLog_result._Fields>, java.io.Serializable, Cloneable, Comparable<replicateLog_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("replicateLog_result");
-
-    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.I64, (short)0);
-    private static final org.apache.thrift.protocol.TField E_FIELD_DESC = new org.apache.thrift.protocol.TField("e", org.apache.thrift.protocol.TType.STRUCT, (short)1);
-
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new replicateLog_resultStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new replicateLog_resultTupleSchemeFactory();
-
-    public long success; // required
-    public @org.apache.thrift.annotation.Nullable RemoteReplicationException e; // 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"),
-      E((short)1, "e");
-
-      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: // E
-            return E;
-          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;
-      }
-
-      @Override
-      public short getThriftFieldId() {
-        return _thriftId;
-      }
-
-      @Override
-      public java.lang.String getFieldName() {
-        return _fieldName;
-      }
-    }
-
-    // isset id assignments
-    private static final int __SUCCESS_ISSET_ID = 0;
-    private byte __isset_bitfield = 0;
-    public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
-    static {
-      java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
-      tmpMap.put(_Fields.E, new org.apache.thrift.meta_data.FieldMetaData("e", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, RemoteReplicationException.class)));
-      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(replicateLog_result.class, metaDataMap);
-    }
-
-    public replicateLog_result() {
-    }
-
-    public replicateLog_result(
-      long success,
-      RemoteReplicationException e)
-    {
-      this();
-      this.success = success;
-      setSuccessIsSet(true);
-      this.e = e;
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public replicateLog_result(replicateLog_result other) {
-      __isset_bitfield = other.__isset_bitfield;
-      this.success = other.success;
-      if (other.isSetE()) {
-        this.e = new RemoteReplicationException(other.e);
-      }
-    }
-
-    @Override
-    public replicateLog_result deepCopy() {
-      return new replicateLog_result(this);
-    }
-
-    @Override
-    public void clear() {
-      setSuccessIsSet(false);
-      this.success = 0;
-      this.e = null;
-    }
-
-    public long getSuccess() {
-      return this.success;
-    }
-
-    public replicateLog_result setSuccess(long success) {
-      this.success = success;
-      setSuccessIsSet(true);
-      return this;
-    }
-
-    public void unsetSuccess() {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __SUCCESS_ISSET_ID);
-    }
-
-    /** Returns true if field success is set (has been assigned a value) and false otherwise */
-    public boolean isSetSuccess() {
-      return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __SUCCESS_ISSET_ID);
-    }
-
-    public void setSuccessIsSet(boolean value) {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __SUCCESS_ISSET_ID, value);
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public RemoteReplicationException getE() {
-      return this.e;
-    }
-
-    public replicateLog_result setE(@org.apache.thrift.annotation.Nullable RemoteReplicationException e) {
-      this.e = e;
-      return this;
-    }
-
-    public void unsetE() {
-      this.e = null;
-    }
-
-    /** Returns true if field e is set (has been assigned a value) and false otherwise */
-    public boolean isSetE() {
-      return this.e != null;
-    }
-
-    public void setEIsSet(boolean value) {
-      if (!value) {
-        this.e = null;
-      }
-    }
-
-    @Override
-    public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
-      switch (field) {
-      case SUCCESS:
-        if (value == null) {
-          unsetSuccess();
-        } else {
-          setSuccess((java.lang.Long)value);
-        }
-        break;
-
-      case E:
-        if (value == null) {
-          unsetE();
-        } else {
-          setE((RemoteReplicationException)value);
-        }
-        break;
-
-      }
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    @Override
-    public java.lang.Object getFieldValue(_Fields field) {
-      switch (field) {
-      case SUCCESS:
-        return getSuccess();
-
-      case E:
-        return getE();
-
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
-    @Override
-    public boolean isSet(_Fields field) {
-      if (field == null) {
-        throw new java.lang.IllegalArgumentException();
-      }
-
-      switch (field) {
-      case SUCCESS:
-        return isSetSuccess();
-      case E:
-        return isSetE();
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(java.lang.Object that) {
-      if (that instanceof replicateLog_result)
-        return this.equals((replicateLog_result)that);
-      return false;
-    }
-
-    public boolean equals(replicateLog_result that) {
-      if (that == null)
-        return false;
-      if (this == that)
-        return true;
-
-      boolean this_present_success = true;
-      boolean that_present_success = true;
-      if (this_present_success || that_present_success) {
-        if (!(this_present_success && that_present_success))
-          return false;
-        if (this.success != that.success)
-          return false;
-      }
-
-      boolean this_present_e = true && this.isSetE();
-      boolean that_present_e = true && that.isSetE();
-      if (this_present_e || that_present_e) {
-        if (!(this_present_e && that_present_e))
-          return false;
-        if (!this.e.equals(that.e))
-          return false;
-      }
-
-      return true;
-    }
-
-    @Override
-    public int hashCode() {
-      int hashCode = 1;
-
-      hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(success);
-
-      hashCode = hashCode * 8191 + ((isSetE()) ? 131071 : 524287);
-      if (isSetE())
-        hashCode = hashCode * 8191 + e.hashCode();
-
-      return hashCode;
-    }
-
-    @Override
-    public int compareTo(replicateLog_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(isSetE(), other.isSetE());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetE()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.e, other.e);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      return 0;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    @Override
-    public _Fields fieldForId(int fieldId) {
-      return _Fields.findByThriftId(fieldId);
-    }
-
-    @Override
-    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("replicateLog_result(");
-      boolean first = true;
-
-      sb.append("success:");
-      sb.append(this.success);
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("e:");
-      if (this.e == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.e);
-      }
-      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 {
-        // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
-        __isset_bitfield = 0;
-        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
-      } catch (org.apache.thrift.TException te) {
-        throw new java.io.IOException(te);
-      }
-    }
-
-    private static class replicateLog_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public replicateLog_resultStandardScheme getScheme() {
-        return new replicateLog_resultStandardScheme();
-      }
-    }
-
-    private static class replicateLog_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<replicateLog_result> {
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol iprot, replicateLog_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.I64) {
-                struct.success = iprot.readI64();
-                struct.setSuccessIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 1: // E
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.e = new RemoteReplicationException();
-                struct.e.read(iprot);
-                struct.setEIsSet(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();
-      }
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol oprot, replicateLog_result struct) throws org.apache.thrift.TException {
-        struct.validate();
-
-        oprot.writeStructBegin(STRUCT_DESC);
-        if (struct.isSetSuccess()) {
-          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
-          oprot.writeI64(struct.success);
-          oprot.writeFieldEnd();
-        }
-        if (struct.e != null) {
-          oprot.writeFieldBegin(E_FIELD_DESC);
-          struct.e.write(oprot);
-          oprot.writeFieldEnd();
-        }
-        oprot.writeFieldStop();
-        oprot.writeStructEnd();
-      }
-
-    }
-
-    private static class replicateLog_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public replicateLog_resultTupleScheme getScheme() {
-        return new replicateLog_resultTupleScheme();
-      }
-    }
-
-    private static class replicateLog_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<replicateLog_result> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, replicateLog_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.isSetE()) {
-          optionals.set(1);
-        }
-        oprot.writeBitSet(optionals, 2);
-        if (struct.isSetSuccess()) {
-          oprot.writeI64(struct.success);
-        }
-        if (struct.isSetE()) {
-          struct.e.write(oprot);
-        }
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, replicateLog_result struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
-        java.util.BitSet incoming = iprot.readBitSet(2);
-        if (incoming.get(0)) {
-          struct.success = iprot.readI64();
-          struct.setSuccessIsSet(true);
-        }
-        if (incoming.get(1)) {
-          struct.e = new RemoteReplicationException();
-          struct.e.read(iprot);
-          struct.setEIsSet(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();
-    }
-  }
-
-  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-  public static class replicateKeyValues_args implements org.apache.thrift.TBase<replicateKeyValues_args, replicateKeyValues_args._Fields>, java.io.Serializable, Cloneable, Comparable<replicateKeyValues_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("replicateKeyValues_args");
-
-    private static final org.apache.thrift.protocol.TField REMOTE_TABLE_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("remoteTableId", org.apache.thrift.protocol.TType.STRING, (short)1);
-    private static final org.apache.thrift.protocol.TField DATA_FIELD_DESC = new org.apache.thrift.protocol.TField("data", 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.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new replicateKeyValues_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new replicateKeyValues_argsTupleSchemeFactory();
-
-    public @org.apache.thrift.annotation.Nullable java.lang.String remoteTableId; // required
-    public @org.apache.thrift.annotation.Nullable KeyValues data; // 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 {
-      REMOTE_TABLE_ID((short)1, "remoteTableId"),
-      DATA((short)2, "data"),
-      CREDENTIALS((short)3, "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: // REMOTE_TABLE_ID
-            return REMOTE_TABLE_ID;
-          case 2: // DATA
-            return DATA;
-          case 3: // 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;
-      }
-
-      @Override
-      public short getThriftFieldId() {
-        return _thriftId;
-      }
-
-      @Override
-      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.REMOTE_TABLE_ID, new org.apache.thrift.meta_data.FieldMetaData("remoteTableId", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
-      tmpMap.put(_Fields.DATA, new org.apache.thrift.meta_data.FieldMetaData("data", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, KeyValues.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(replicateKeyValues_args.class, metaDataMap);
-    }
-
-    public replicateKeyValues_args() {
-    }
-
-    public replicateKeyValues_args(
-      java.lang.String remoteTableId,
-      KeyValues data,
-      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials)
-    {
-      this();
-      this.remoteTableId = remoteTableId;
-      this.data = data;
-      this.credentials = credentials;
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public replicateKeyValues_args(replicateKeyValues_args other) {
-      if (other.isSetRemoteTableId()) {
-        this.remoteTableId = other.remoteTableId;
-      }
-      if (other.isSetData()) {
-        this.data = new KeyValues(other.data);
-      }
-      if (other.isSetCredentials()) {
-        this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
-      }
-    }
-
-    @Override
-    public replicateKeyValues_args deepCopy() {
-      return new replicateKeyValues_args(this);
-    }
-
-    @Override
-    public void clear() {
-      this.remoteTableId = null;
-      this.data = null;
-      this.credentials = null;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.lang.String getRemoteTableId() {
-      return this.remoteTableId;
-    }
-
-    public replicateKeyValues_args setRemoteTableId(@org.apache.thrift.annotation.Nullable java.lang.String remoteTableId) {
-      this.remoteTableId = remoteTableId;
-      return this;
-    }
-
-    public void unsetRemoteTableId() {
-      this.remoteTableId = null;
-    }
-
-    /** Returns true if field remoteTableId is set (has been assigned a value) and false otherwise */
-    public boolean isSetRemoteTableId() {
-      return this.remoteTableId != null;
-    }
-
-    public void setRemoteTableIdIsSet(boolean value) {
-      if (!value) {
-        this.remoteTableId = null;
-      }
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public KeyValues getData() {
-      return this.data;
-    }
-
-    public replicateKeyValues_args setData(@org.apache.thrift.annotation.Nullable KeyValues data) {
-      this.data = data;
-      return this;
-    }
-
-    public void unsetData() {
-      this.data = null;
-    }
-
-    /** Returns true if field data is set (has been assigned a value) and false otherwise */
-    public boolean isSetData() {
-      return this.data != null;
-    }
-
-    public void setDataIsSet(boolean value) {
-      if (!value) {
-        this.data = null;
-      }
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.securityImpl.thrift.TCredentials getCredentials() {
-      return this.credentials;
-    }
-
-    public replicateKeyValues_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;
-      }
-    }
-
-    @Override
-    public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
-      switch (field) {
-      case REMOTE_TABLE_ID:
-        if (value == null) {
-          unsetRemoteTableId();
-        } else {
-          setRemoteTableId((java.lang.String)value);
-        }
-        break;
-
-      case DATA:
-        if (value == null) {
-          unsetData();
-        } else {
-          setData((KeyValues)value);
-        }
-        break;
-
-      case CREDENTIALS:
-        if (value == null) {
-          unsetCredentials();
-        } else {
-          setCredentials((org.apache.accumulo.core.securityImpl.thrift.TCredentials)value);
-        }
-        break;
-
-      }
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    @Override
-    public java.lang.Object getFieldValue(_Fields field) {
-      switch (field) {
-      case REMOTE_TABLE_ID:
-        return getRemoteTableId();
-
-      case DATA:
-        return getData();
-
-      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 */
-    @Override
-    public boolean isSet(_Fields field) {
-      if (field == null) {
-        throw new java.lang.IllegalArgumentException();
-      }
-
-      switch (field) {
-      case REMOTE_TABLE_ID:
-        return isSetRemoteTableId();
-      case DATA:
-        return isSetData();
-      case CREDENTIALS:
-        return isSetCredentials();
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(java.lang.Object that) {
-      if (that instanceof replicateKeyValues_args)
-        return this.equals((replicateKeyValues_args)that);
-      return false;
-    }
-
-    public boolean equals(replicateKeyValues_args that) {
-      if (that == null)
-        return false;
-      if (this == that)
-        return true;
-
-      boolean this_present_remoteTableId = true && this.isSetRemoteTableId();
-      boolean that_present_remoteTableId = true && that.isSetRemoteTableId();
-      if (this_present_remoteTableId || that_present_remoteTableId) {
-        if (!(this_present_remoteTableId && that_present_remoteTableId))
-          return false;
-        if (!this.remoteTableId.equals(that.remoteTableId))
-          return false;
-      }
-
-      boolean this_present_data = true && this.isSetData();
-      boolean that_present_data = true && that.isSetData();
-      if (this_present_data || that_present_data) {
-        if (!(this_present_data && that_present_data))
-          return false;
-        if (!this.data.equals(that.data))
-          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 + ((isSetRemoteTableId()) ? 131071 : 524287);
-      if (isSetRemoteTableId())
-        hashCode = hashCode * 8191 + remoteTableId.hashCode();
-
-      hashCode = hashCode * 8191 + ((isSetData()) ? 131071 : 524287);
-      if (isSetData())
-        hashCode = hashCode * 8191 + data.hashCode();
-
-      hashCode = hashCode * 8191 + ((isSetCredentials()) ? 131071 : 524287);
-      if (isSetCredentials())
-        hashCode = hashCode * 8191 + credentials.hashCode();
-
-      return hashCode;
-    }
-
-    @Override
-    public int compareTo(replicateKeyValues_args other) {
-      if (!getClass().equals(other.getClass())) {
-        return getClass().getName().compareTo(other.getClass().getName());
-      }
-
-      int lastComparison = 0;
-
-      lastComparison = java.lang.Boolean.compare(isSetRemoteTableId(), other.isSetRemoteTableId());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetRemoteTableId()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.remoteTableId, other.remoteTableId);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = java.lang.Boolean.compare(isSetData(), other.isSetData());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetData()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.data, other.data);
-        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
-    @Override
-    public _Fields fieldForId(int fieldId) {
-      return _Fields.findByThriftId(fieldId);
-    }
-
-    @Override
-    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
-      scheme(iprot).read(iprot, this);
-    }
-
-    @Override
-    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("replicateKeyValues_args(");
-      boolean first = true;
-
-      sb.append("remoteTableId:");
-      if (this.remoteTableId == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.remoteTableId);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("data:");
-      if (this.data == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.data);
-      }
-      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 (data != null) {
-        data.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 replicateKeyValues_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public replicateKeyValues_argsStandardScheme getScheme() {
-        return new replicateKeyValues_argsStandardScheme();
-      }
-    }
-
-    private static class replicateKeyValues_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<replicateKeyValues_args> {
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol iprot, replicateKeyValues_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: // REMOTE_TABLE_ID
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
-                struct.remoteTableId = iprot.readString();
-                struct.setRemoteTableIdIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 2: // DATA
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.data = new KeyValues();
-                struct.data.read(iprot);
-                struct.setDataIsSet(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;
-            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();
-      }
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol oprot, replicateKeyValues_args struct) throws org.apache.thrift.TException {
-        struct.validate();
-
-        oprot.writeStructBegin(STRUCT_DESC);
-        if (struct.remoteTableId != null) {
-          oprot.writeFieldBegin(REMOTE_TABLE_ID_FIELD_DESC);
-          oprot.writeString(struct.remoteTableId);
-          oprot.writeFieldEnd();
-        }
-        if (struct.data != null) {
-          oprot.writeFieldBegin(DATA_FIELD_DESC);
-          struct.data.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 replicateKeyValues_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public replicateKeyValues_argsTupleScheme getScheme() {
-        return new replicateKeyValues_argsTupleScheme();
-      }
-    }
-
-    private static class replicateKeyValues_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<replicateKeyValues_args> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, replicateKeyValues_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.isSetRemoteTableId()) {
-          optionals.set(0);
-        }
-        if (struct.isSetData()) {
-          optionals.set(1);
-        }
-        if (struct.isSetCredentials()) {
-          optionals.set(2);
-        }
-        oprot.writeBitSet(optionals, 3);
-        if (struct.isSetRemoteTableId()) {
-          oprot.writeString(struct.remoteTableId);
-        }
-        if (struct.isSetData()) {
-          struct.data.write(oprot);
-        }
-        if (struct.isSetCredentials()) {
-          struct.credentials.write(oprot);
-        }
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, replicateKeyValues_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.remoteTableId = iprot.readString();
-          struct.setRemoteTableIdIsSet(true);
-        }
-        if (incoming.get(1)) {
-          struct.data = new KeyValues();
-          struct.data.read(iprot);
-          struct.setDataIsSet(true);
-        }
-        if (incoming.get(2)) {
-          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();
-    }
-  }
-
-  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-  public static class replicateKeyValues_result implements org.apache.thrift.TBase<replicateKeyValues_result, replicateKeyValues_result._Fields>, java.io.Serializable, Cloneable, Comparable<replicateKeyValues_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("replicateKeyValues_result");
-
-    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.I64, (short)0);
-    private static final org.apache.thrift.protocol.TField E_FIELD_DESC = new org.apache.thrift.protocol.TField("e", org.apache.thrift.protocol.TType.STRUCT, (short)1);
-
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new replicateKeyValues_resultStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new replicateKeyValues_resultTupleSchemeFactory();
-
-    public long success; // required
-    public @org.apache.thrift.annotation.Nullable RemoteReplicationException e; // 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"),
-      E((short)1, "e");
-
-      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: // E
-            return E;
-          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;
-      }
-
-      @Override
-      public short getThriftFieldId() {
-        return _thriftId;
-      }
-
-      @Override
-      public java.lang.String getFieldName() {
-        return _fieldName;
-      }
-    }
-
-    // isset id assignments
-    private static final int __SUCCESS_ISSET_ID = 0;
-    private byte __isset_bitfield = 0;
-    public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
-    static {
-      java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
-      tmpMap.put(_Fields.E, new org.apache.thrift.meta_data.FieldMetaData("e", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, RemoteReplicationException.class)));
-      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(replicateKeyValues_result.class, metaDataMap);
-    }
-
-    public replicateKeyValues_result() {
-    }
-
-    public replicateKeyValues_result(
-      long success,
-      RemoteReplicationException e)
-    {
-      this();
-      this.success = success;
-      setSuccessIsSet(true);
-      this.e = e;
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public replicateKeyValues_result(replicateKeyValues_result other) {
-      __isset_bitfield = other.__isset_bitfield;
-      this.success = other.success;
-      if (other.isSetE()) {
-        this.e = new RemoteReplicationException(other.e);
-      }
-    }
-
-    @Override
-    public replicateKeyValues_result deepCopy() {
-      return new replicateKeyValues_result(this);
-    }
-
-    @Override
-    public void clear() {
-      setSuccessIsSet(false);
-      this.success = 0;
-      this.e = null;
-    }
-
-    public long getSuccess() {
-      return this.success;
-    }
-
-    public replicateKeyValues_result setSuccess(long success) {
-      this.success = success;
-      setSuccessIsSet(true);
-      return this;
-    }
-
-    public void unsetSuccess() {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __SUCCESS_ISSET_ID);
-    }
-
-    /** Returns true if field success is set (has been assigned a value) and false otherwise */
-    public boolean isSetSuccess() {
-      return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __SUCCESS_ISSET_ID);
-    }
-
-    public void setSuccessIsSet(boolean value) {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __SUCCESS_ISSET_ID, value);
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public RemoteReplicationException getE() {
-      return this.e;
-    }
-
-    public replicateKeyValues_result setE(@org.apache.thrift.annotation.Nullable RemoteReplicationException e) {
-      this.e = e;
-      return this;
-    }
-
-    public void unsetE() {
-      this.e = null;
-    }
-
-    /** Returns true if field e is set (has been assigned a value) and false otherwise */
-    public boolean isSetE() {
-      return this.e != null;
-    }
-
-    public void setEIsSet(boolean value) {
-      if (!value) {
-        this.e = null;
-      }
-    }
-
-    @Override
-    public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
-      switch (field) {
-      case SUCCESS:
-        if (value == null) {
-          unsetSuccess();
-        } else {
-          setSuccess((java.lang.Long)value);
-        }
-        break;
-
-      case E:
-        if (value == null) {
-          unsetE();
-        } else {
-          setE((RemoteReplicationException)value);
-        }
-        break;
-
-      }
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    @Override
-    public java.lang.Object getFieldValue(_Fields field) {
-      switch (field) {
-      case SUCCESS:
-        return getSuccess();
-
-      case E:
-        return getE();
-
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
-    @Override
-    public boolean isSet(_Fields field) {
-      if (field == null) {
-        throw new java.lang.IllegalArgumentException();
-      }
-
-      switch (field) {
-      case SUCCESS:
-        return isSetSuccess();
-      case E:
-        return isSetE();
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(java.lang.Object that) {
-      if (that instanceof replicateKeyValues_result)
-        return this.equals((replicateKeyValues_result)that);
-      return false;
-    }
-
-    public boolean equals(replicateKeyValues_result that) {
-      if (that == null)
-        return false;
-      if (this == that)
-        return true;
-
-      boolean this_present_success = true;
-      boolean that_present_success = true;
-      if (this_present_success || that_present_success) {
-        if (!(this_present_success && that_present_success))
-          return false;
-        if (this.success != that.success)
-          return false;
-      }
-
-      boolean this_present_e = true && this.isSetE();
-      boolean that_present_e = true && that.isSetE();
-      if (this_present_e || that_present_e) {
-        if (!(this_present_e && that_present_e))
-          return false;
-        if (!this.e.equals(that.e))
-          return false;
-      }
-
-      return true;
-    }
-
-    @Override
-    public int hashCode() {
-      int hashCode = 1;
-
-      hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(success);
-
-      hashCode = hashCode * 8191 + ((isSetE()) ? 131071 : 524287);
-      if (isSetE())
-        hashCode = hashCode * 8191 + e.hashCode();
-
-      return hashCode;
-    }
-
-    @Override
-    public int compareTo(replicateKeyValues_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(isSetE(), other.isSetE());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetE()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.e, other.e);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      return 0;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    @Override
-    public _Fields fieldForId(int fieldId) {
-      return _Fields.findByThriftId(fieldId);
-    }
-
-    @Override
-    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("replicateKeyValues_result(");
-      boolean first = true;
-
-      sb.append("success:");
-      sb.append(this.success);
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("e:");
-      if (this.e == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.e);
-      }
-      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 {
-        // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
-        __isset_bitfield = 0;
-        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
-      } catch (org.apache.thrift.TException te) {
-        throw new java.io.IOException(te);
-      }
-    }
-
-    private static class replicateKeyValues_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public replicateKeyValues_resultStandardScheme getScheme() {
-        return new replicateKeyValues_resultStandardScheme();
-      }
-    }
-
-    private static class replicateKeyValues_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<replicateKeyValues_result> {
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol iprot, replicateKeyValues_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.I64) {
-                struct.success = iprot.readI64();
-                struct.setSuccessIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 1: // E
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.e = new RemoteReplicationException();
-                struct.e.read(iprot);
-                struct.setEIsSet(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();
-      }
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol oprot, replicateKeyValues_result struct) throws org.apache.thrift.TException {
-        struct.validate();
-
-        oprot.writeStructBegin(STRUCT_DESC);
-        if (struct.isSetSuccess()) {
-          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
-          oprot.writeI64(struct.success);
-          oprot.writeFieldEnd();
-        }
-        if (struct.e != null) {
-          oprot.writeFieldBegin(E_FIELD_DESC);
-          struct.e.write(oprot);
-          oprot.writeFieldEnd();
-        }
-        oprot.writeFieldStop();
-        oprot.writeStructEnd();
-      }
-
-    }
-
-    private static class replicateKeyValues_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public replicateKeyValues_resultTupleScheme getScheme() {
-        return new replicateKeyValues_resultTupleScheme();
-      }
-    }
-
-    private static class replicateKeyValues_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<replicateKeyValues_result> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, replicateKeyValues_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.isSetE()) {
-          optionals.set(1);
-        }
-        oprot.writeBitSet(optionals, 2);
-        if (struct.isSetSuccess()) {
-          oprot.writeI64(struct.success);
-        }
-        if (struct.isSetE()) {
-          struct.e.write(oprot);
-        }
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, replicateKeyValues_result struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
-        java.util.BitSet incoming = iprot.readBitSet(2);
-        if (incoming.get(0)) {
-          struct.success = iprot.readI64();
-          struct.setSuccessIsSet(true);
-        }
-        if (incoming.get(1)) {
-          struct.e = new RemoteReplicationException();
-          struct.e.read(iprot);
-          struct.setEIsSet(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();
-    }
-  }
-
-  private static void unusedMethod() {}
-}
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/replication/thrift/WalEdits.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/replication/thrift/WalEdits.java
deleted file mode 100644
index 48df668..0000000
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/replication/thrift/WalEdits.java
+++ /dev/null
@@ -1,456 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/**
- * Autogenerated by Thrift Compiler (0.17.0)
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- *  @generated
- */
-package org.apache.accumulo.core.replication.thrift;
-
-@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-public class WalEdits implements org.apache.thrift.TBase<WalEdits, WalEdits._Fields>, java.io.Serializable, Cloneable, Comparable<WalEdits> {
-  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("WalEdits");
-
-  private static final org.apache.thrift.protocol.TField EDITS_FIELD_DESC = new org.apache.thrift.protocol.TField("edits", org.apache.thrift.protocol.TType.LIST, (short)1);
-
-  private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new WalEditsStandardSchemeFactory();
-  private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new WalEditsTupleSchemeFactory();
-
-  public @org.apache.thrift.annotation.Nullable java.util.List<java.nio.ByteBuffer> edits; // 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 {
-    EDITS((short)1, "edits");
-
-    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: // EDITS
-          return EDITS;
-        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;
-    }
-
-    @Override
-    public short getThriftFieldId() {
-      return _thriftId;
-    }
-
-    @Override
-    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.EDITS, new org.apache.thrift.meta_data.FieldMetaData("edits", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
-            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING            , true))));
-    metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(WalEdits.class, metaDataMap);
-  }
-
-  public WalEdits() {
-  }
-
-  public WalEdits(
-    java.util.List<java.nio.ByteBuffer> edits)
-  {
-    this();
-    this.edits = edits;
-  }
-
-  /**
-   * Performs a deep copy on <i>other</i>.
-   */
-  public WalEdits(WalEdits other) {
-    if (other.isSetEdits()) {
-      java.util.List<java.nio.ByteBuffer> __this__edits = new java.util.ArrayList<java.nio.ByteBuffer>(other.edits);
-      this.edits = __this__edits;
-    }
-  }
-
-  @Override
-  public WalEdits deepCopy() {
-    return new WalEdits(this);
-  }
-
-  @Override
-  public void clear() {
-    this.edits = null;
-  }
-
-  public int getEditsSize() {
-    return (this.edits == null) ? 0 : this.edits.size();
-  }
-
-  @org.apache.thrift.annotation.Nullable
-  public java.util.Iterator<java.nio.ByteBuffer> getEditsIterator() {
-    return (this.edits == null) ? null : this.edits.iterator();
-  }
-
-  public void addToEdits(java.nio.ByteBuffer elem) {
-    if (this.edits == null) {
-      this.edits = new java.util.ArrayList<java.nio.ByteBuffer>();
-    }
-    this.edits.add(elem);
-  }
-
-  @org.apache.thrift.annotation.Nullable
-  public java.util.List<java.nio.ByteBuffer> getEdits() {
-    return this.edits;
-  }
-
-  public WalEdits setEdits(@org.apache.thrift.annotation.Nullable java.util.List<java.nio.ByteBuffer> edits) {
-    this.edits = edits;
-    return this;
-  }
-
-  public void unsetEdits() {
-    this.edits = null;
-  }
-
-  /** Returns true if field edits is set (has been assigned a value) and false otherwise */
-  public boolean isSetEdits() {
-    return this.edits != null;
-  }
-
-  public void setEditsIsSet(boolean value) {
-    if (!value) {
-      this.edits = null;
-    }
-  }
-
-  @Override
-  public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
-    switch (field) {
-    case EDITS:
-      if (value == null) {
-        unsetEdits();
-      } else {
-        setEdits((java.util.List<java.nio.ByteBuffer>)value);
-      }
-      break;
-
-    }
-  }
-
-  @org.apache.thrift.annotation.Nullable
-  @Override
-  public java.lang.Object getFieldValue(_Fields field) {
-    switch (field) {
-    case EDITS:
-      return getEdits();
-
-    }
-    throw new java.lang.IllegalStateException();
-  }
-
-  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
-  @Override
-  public boolean isSet(_Fields field) {
-    if (field == null) {
-      throw new java.lang.IllegalArgumentException();
-    }
-
-    switch (field) {
-    case EDITS:
-      return isSetEdits();
-    }
-    throw new java.lang.IllegalStateException();
-  }
-
-  @Override
-  public boolean equals(java.lang.Object that) {
-    if (that instanceof WalEdits)
-      return this.equals((WalEdits)that);
-    return false;
-  }
-
-  public boolean equals(WalEdits that) {
-    if (that == null)
-      return false;
-    if (this == that)
-      return true;
-
-    boolean this_present_edits = true && this.isSetEdits();
-    boolean that_present_edits = true && that.isSetEdits();
-    if (this_present_edits || that_present_edits) {
-      if (!(this_present_edits && that_present_edits))
-        return false;
-      if (!this.edits.equals(that.edits))
-        return false;
-    }
-
-    return true;
-  }
-
-  @Override
-  public int hashCode() {
-    int hashCode = 1;
-
-    hashCode = hashCode * 8191 + ((isSetEdits()) ? 131071 : 524287);
-    if (isSetEdits())
-      hashCode = hashCode * 8191 + edits.hashCode();
-
-    return hashCode;
-  }
-
-  @Override
-  public int compareTo(WalEdits other) {
-    if (!getClass().equals(other.getClass())) {
-      return getClass().getName().compareTo(other.getClass().getName());
-    }
-
-    int lastComparison = 0;
-
-    lastComparison = java.lang.Boolean.compare(isSetEdits(), other.isSetEdits());
-    if (lastComparison != 0) {
-      return lastComparison;
-    }
-    if (isSetEdits()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.edits, other.edits);
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-    }
-    return 0;
-  }
-
-  @org.apache.thrift.annotation.Nullable
-  @Override
-  public _Fields fieldForId(int fieldId) {
-    return _Fields.findByThriftId(fieldId);
-  }
-
-  @Override
-  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
-    scheme(iprot).read(iprot, this);
-  }
-
-  @Override
-  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("WalEdits(");
-    boolean first = true;
-
-    sb.append("edits:");
-    if (this.edits == null) {
-      sb.append("null");
-    } else {
-      org.apache.thrift.TBaseHelper.toString(this.edits, sb);
-    }
-    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 WalEditsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-    @Override
-    public WalEditsStandardScheme getScheme() {
-      return new WalEditsStandardScheme();
-    }
-  }
-
-  private static class WalEditsStandardScheme extends org.apache.thrift.scheme.StandardScheme<WalEdits> {
-
-    @Override
-    public void read(org.apache.thrift.protocol.TProtocol iprot, WalEdits 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: // EDITS
-            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
-              {
-                org.apache.thrift.protocol.TList _list0 = iprot.readListBegin();
-                struct.edits = new java.util.ArrayList<java.nio.ByteBuffer>(_list0.size);
-                @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem1;
-                for (int _i2 = 0; _i2 < _list0.size; ++_i2)
-                {
-                  _elem1 = iprot.readBinary();
-                  struct.edits.add(_elem1);
-                }
-                iprot.readListEnd();
-              }
-              struct.setEditsIsSet(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();
-    }
-
-    @Override
-    public void write(org.apache.thrift.protocol.TProtocol oprot, WalEdits struct) throws org.apache.thrift.TException {
-      struct.validate();
-
-      oprot.writeStructBegin(STRUCT_DESC);
-      if (struct.edits != null) {
-        oprot.writeFieldBegin(EDITS_FIELD_DESC);
-        {
-          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.edits.size()));
-          for (java.nio.ByteBuffer _iter3 : struct.edits)
-          {
-            oprot.writeBinary(_iter3);
-          }
-          oprot.writeListEnd();
-        }
-        oprot.writeFieldEnd();
-      }
-      oprot.writeFieldStop();
-      oprot.writeStructEnd();
-    }
-
-  }
-
-  private static class WalEditsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-    @Override
-    public WalEditsTupleScheme getScheme() {
-      return new WalEditsTupleScheme();
-    }
-  }
-
-  private static class WalEditsTupleScheme extends org.apache.thrift.scheme.TupleScheme<WalEdits> {
-
-    @Override
-    public void write(org.apache.thrift.protocol.TProtocol prot, WalEdits 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.isSetEdits()) {
-        optionals.set(0);
-      }
-      oprot.writeBitSet(optionals, 1);
-      if (struct.isSetEdits()) {
-        {
-          oprot.writeI32(struct.edits.size());
-          for (java.nio.ByteBuffer _iter4 : struct.edits)
-          {
-            oprot.writeBinary(_iter4);
-          }
-        }
-      }
-    }
-
-    @Override
-    public void read(org.apache.thrift.protocol.TProtocol prot, WalEdits 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.TList _list5 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
-          struct.edits = new java.util.ArrayList<java.nio.ByteBuffer>(_list5.size);
-          @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem6;
-          for (int _i7 = 0; _i7 < _list5.size; ++_i7)
-          {
-            _elem6 = iprot.readBinary();
-            struct.edits.add(_elem6);
-          }
-        }
-        struct.setEditsIsSet(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();
-  }
-  private static void unusedMethod() {}
-}
-
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TUnloadTabletGoal.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tablet/thrift/TUnloadTabletGoal.java
similarity index 96%
rename from core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TUnloadTabletGoal.java
rename to core/src/main/thrift-gen-java/org/apache/accumulo/core/tablet/thrift/TUnloadTabletGoal.java
index 4c65ad9..747f4ee 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TUnloadTabletGoal.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tablet/thrift/TUnloadTabletGoal.java
@@ -22,7 +22,7 @@
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
  */
-package org.apache.accumulo.core.tabletserver.thrift;
+package org.apache.accumulo.core.tablet.thrift;
 
 
 public enum TUnloadTabletGoal implements org.apache.thrift.TEnum {
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tablet/thrift/TabletManagementClientService.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tablet/thrift/TabletManagementClientService.java
new file mode 100644
index 0000000..6b83212
--- /dev/null
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tablet/thrift/TabletManagementClientService.java
@@ -0,0 +1,4267 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.17.0)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.accumulo.core.tablet.thrift;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+public class TabletManagementClientService {
+
+  public interface Iface {
+
+    public void loadTablet(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent) throws org.apache.thrift.TException;
+
+    public void unloadTablet(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, TUnloadTabletGoal goal, long requestTime) throws org.apache.thrift.TException;
+
+    public void splitTablet(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, java.nio.ByteBuffer splitPoint) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException, org.apache.thrift.TException;
+
+    public void flushTablet(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent) throws org.apache.thrift.TException;
+
+  }
+
+  public interface AsyncIface {
+
+    public void loadTablet(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+
+    public void unloadTablet(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, TUnloadTabletGoal goal, long requestTime, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+
+    public void splitTablet(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, java.nio.ByteBuffer splitPoint, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+
+    public void flushTablet(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+
+  }
+
+  public static class Client extends org.apache.thrift.TServiceClient implements Iface {
+    public static class Factory implements org.apache.thrift.TServiceClientFactory<Client> {
+      public Factory() {}
+      @Override
+      public Client getClient(org.apache.thrift.protocol.TProtocol prot) {
+        return new Client(prot);
+      }
+      @Override
+      public Client getClient(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TProtocol oprot) {
+        return new Client(iprot, oprot);
+      }
+    }
+
+    public Client(org.apache.thrift.protocol.TProtocol prot)
+    {
+      super(prot, prot);
+    }
+
+    public Client(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TProtocol oprot) {
+      super(iprot, oprot);
+    }
+
+    @Override
+    public void loadTablet(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent) throws org.apache.thrift.TException
+    {
+      send_loadTablet(tinfo, credentials, lock, extent);
+    }
+
+    public void send_loadTablet(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent) throws org.apache.thrift.TException
+    {
+      loadTablet_args args = new loadTablet_args();
+      args.setTinfo(tinfo);
+      args.setCredentials(credentials);
+      args.setLock(lock);
+      args.setExtent(extent);
+      sendBaseOneway("loadTablet", args);
+    }
+
+    @Override
+    public void unloadTablet(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, TUnloadTabletGoal goal, long requestTime) throws org.apache.thrift.TException
+    {
+      send_unloadTablet(tinfo, credentials, lock, extent, goal, requestTime);
+    }
+
+    public void send_unloadTablet(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, TUnloadTabletGoal goal, long requestTime) throws org.apache.thrift.TException
+    {
+      unloadTablet_args args = new unloadTablet_args();
+      args.setTinfo(tinfo);
+      args.setCredentials(credentials);
+      args.setLock(lock);
+      args.setExtent(extent);
+      args.setGoal(goal);
+      args.setRequestTime(requestTime);
+      sendBaseOneway("unloadTablet", args);
+    }
+
+    @Override
+    public void splitTablet(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, java.nio.ByteBuffer splitPoint) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException, org.apache.thrift.TException
+    {
+      send_splitTablet(tinfo, credentials, extent, splitPoint);
+      recv_splitTablet();
+    }
+
+    public void send_splitTablet(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, java.nio.ByteBuffer splitPoint) throws org.apache.thrift.TException
+    {
+      splitTablet_args args = new splitTablet_args();
+      args.setTinfo(tinfo);
+      args.setCredentials(credentials);
+      args.setExtent(extent);
+      args.setSplitPoint(splitPoint);
+      sendBase("splitTablet", args);
+    }
+
+    public void recv_splitTablet() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException, org.apache.thrift.TException
+    {
+      splitTablet_result result = new splitTablet_result();
+      receiveBase(result, "splitTablet");
+      if (result.sec != null) {
+        throw result.sec;
+      }
+      if (result.nste != null) {
+        throw result.nste;
+      }
+      return;
+    }
+
+    @Override
+    public void flushTablet(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent) throws org.apache.thrift.TException
+    {
+      send_flushTablet(tinfo, credentials, lock, extent);
+    }
+
+    public void send_flushTablet(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent) throws org.apache.thrift.TException
+    {
+      flushTablet_args args = new flushTablet_args();
+      args.setTinfo(tinfo);
+      args.setCredentials(credentials);
+      args.setLock(lock);
+      args.setExtent(extent);
+      sendBaseOneway("flushTablet", args);
+    }
+
+  }
+  public static class AsyncClient extends org.apache.thrift.async.TAsyncClient implements AsyncIface {
+    public static class Factory implements org.apache.thrift.async.TAsyncClientFactory<AsyncClient> {
+      private org.apache.thrift.async.TAsyncClientManager clientManager;
+      private org.apache.thrift.protocol.TProtocolFactory protocolFactory;
+      public Factory(org.apache.thrift.async.TAsyncClientManager clientManager, org.apache.thrift.protocol.TProtocolFactory protocolFactory) {
+        this.clientManager = clientManager;
+        this.protocolFactory = protocolFactory;
+      }
+    @Override
+      public AsyncClient getAsyncClient(org.apache.thrift.transport.TNonblockingTransport transport) {
+        return new AsyncClient(protocolFactory, clientManager, transport);
+      }
+    }
+
+    public AsyncClient(org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.async.TAsyncClientManager clientManager, org.apache.thrift.transport.TNonblockingTransport transport) {
+      super(protocolFactory, clientManager, transport);
+    }
+
+    @Override
+    public void loadTablet(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      loadTablet_call method_call = new loadTablet_call(tinfo, credentials, lock, extent, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class loadTablet_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
+      private java.lang.String lock;
+      private org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent;
+      public loadTablet_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.thrift.async.AsyncMethodCallback<Void> 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, true);
+        this.tinfo = tinfo;
+        this.credentials = credentials;
+        this.lock = lock;
+        this.extent = extent;
+      }
+
+      @Override
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("loadTablet", org.apache.thrift.protocol.TMessageType.ONEWAY, 0));
+        loadTablet_args args = new loadTablet_args();
+        args.setTinfo(tinfo);
+        args.setCredentials(credentials);
+        args.setLock(lock);
+        args.setExtent(extent);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      @Override
+      public Void 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 null;
+      }
+    }
+
+    @Override
+    public void unloadTablet(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, TUnloadTabletGoal goal, long requestTime, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      unloadTablet_call method_call = new unloadTablet_call(tinfo, credentials, lock, extent, goal, requestTime, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class unloadTablet_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
+      private java.lang.String lock;
+      private org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent;
+      private TUnloadTabletGoal goal;
+      private long requestTime;
+      public unloadTablet_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, TUnloadTabletGoal goal, long requestTime, org.apache.thrift.async.AsyncMethodCallback<Void> 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, true);
+        this.tinfo = tinfo;
+        this.credentials = credentials;
+        this.lock = lock;
+        this.extent = extent;
+        this.goal = goal;
+        this.requestTime = requestTime;
+      }
+
+      @Override
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("unloadTablet", org.apache.thrift.protocol.TMessageType.ONEWAY, 0));
+        unloadTablet_args args = new unloadTablet_args();
+        args.setTinfo(tinfo);
+        args.setCredentials(credentials);
+        args.setLock(lock);
+        args.setExtent(extent);
+        args.setGoal(goal);
+        args.setRequestTime(requestTime);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      @Override
+      public Void 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 null;
+      }
+    }
+
+    @Override
+    public void splitTablet(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, java.nio.ByteBuffer splitPoint, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      splitTablet_call method_call = new splitTablet_call(tinfo, credentials, extent, splitPoint, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class splitTablet_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
+      private org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent;
+      private java.nio.ByteBuffer splitPoint;
+      public splitTablet_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, java.nio.ByteBuffer splitPoint, org.apache.thrift.async.AsyncMethodCallback<Void> 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.extent = extent;
+        this.splitPoint = splitPoint;
+      }
+
+      @Override
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("splitTablet", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        splitTablet_args args = new splitTablet_args();
+        args.setTinfo(tinfo);
+        args.setCredentials(credentials);
+        args.setExtent(extent);
+        args.setSplitPoint(splitPoint);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      @Override
+      public Void getResult() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException, 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);
+        (new Client(prot)).recv_splitTablet();
+        return null;
+      }
+    }
+
+    @Override
+    public void flushTablet(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      flushTablet_call method_call = new flushTablet_call(tinfo, credentials, lock, extent, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class flushTablet_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
+      private java.lang.String lock;
+      private org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent;
+      public flushTablet_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.thrift.async.AsyncMethodCallback<Void> 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, true);
+        this.tinfo = tinfo;
+        this.credentials = credentials;
+        this.lock = lock;
+        this.extent = extent;
+      }
+
+      @Override
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("flushTablet", org.apache.thrift.protocol.TMessageType.ONEWAY, 0));
+        flushTablet_args args = new flushTablet_args();
+        args.setTinfo(tinfo);
+        args.setCredentials(credentials);
+        args.setLock(lock);
+        args.setExtent(extent);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      @Override
+      public Void 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 null;
+      }
+    }
+
+  }
+
+  public static class Processor<I extends Iface> extends org.apache.thrift.TBaseProcessor<I> implements org.apache.thrift.TProcessor {
+    private static final org.slf4j.Logger _LOGGER = org.slf4j.LoggerFactory.getLogger(Processor.class.getName());
+    public Processor(I iface) {
+      super(iface, getProcessMap(new java.util.HashMap<java.lang.String, org.apache.thrift.ProcessFunction<I, ? extends org.apache.thrift.TBase>>()));
+    }
+
+    protected Processor(I iface, java.util.Map<java.lang.String, org.apache.thrift.ProcessFunction<I, ? extends org.apache.thrift.TBase>> processMap) {
+      super(iface, getProcessMap(processMap));
+    }
+
+    private static <I extends Iface> java.util.Map<java.lang.String,  org.apache.thrift.ProcessFunction<I, ? extends org.apache.thrift.TBase>> getProcessMap(java.util.Map<java.lang.String, org.apache.thrift.ProcessFunction<I, ? extends  org.apache.thrift.TBase>> processMap) {
+      processMap.put("loadTablet", new loadTablet());
+      processMap.put("unloadTablet", new unloadTablet());
+      processMap.put("splitTablet", new splitTablet());
+      processMap.put("flushTablet", new flushTablet());
+      return processMap;
+    }
+
+    public static class loadTablet<I extends Iface> extends org.apache.thrift.ProcessFunction<I, loadTablet_args> {
+      public loadTablet() {
+        super("loadTablet");
+      }
+
+      @Override
+      public loadTablet_args getEmptyArgsInstance() {
+        return new loadTablet_args();
+      }
+
+      @Override
+      protected boolean isOneway() {
+        return true;
+      }
+
+      @Override
+      protected boolean rethrowUnhandledExceptions() {
+        return false;
+      }
+
+      @Override
+      public org.apache.thrift.TBase getResult(I iface, loadTablet_args args) throws org.apache.thrift.TException {
+        iface.loadTablet(args.tinfo, args.credentials, args.lock, args.extent);
+        return null;
+      }
+    }
+
+    public static class unloadTablet<I extends Iface> extends org.apache.thrift.ProcessFunction<I, unloadTablet_args> {
+      public unloadTablet() {
+        super("unloadTablet");
+      }
+
+      @Override
+      public unloadTablet_args getEmptyArgsInstance() {
+        return new unloadTablet_args();
+      }
+
+      @Override
+      protected boolean isOneway() {
+        return true;
+      }
+
+      @Override
+      protected boolean rethrowUnhandledExceptions() {
+        return false;
+      }
+
+      @Override
+      public org.apache.thrift.TBase getResult(I iface, unloadTablet_args args) throws org.apache.thrift.TException {
+        iface.unloadTablet(args.tinfo, args.credentials, args.lock, args.extent, args.goal, args.requestTime);
+        return null;
+      }
+    }
+
+    public static class splitTablet<I extends Iface> extends org.apache.thrift.ProcessFunction<I, splitTablet_args> {
+      public splitTablet() {
+        super("splitTablet");
+      }
+
+      @Override
+      public splitTablet_args getEmptyArgsInstance() {
+        return new splitTablet_args();
+      }
+
+      @Override
+      protected boolean isOneway() {
+        return false;
+      }
+
+      @Override
+      protected boolean rethrowUnhandledExceptions() {
+        return false;
+      }
+
+      @Override
+      public splitTablet_result getResult(I iface, splitTablet_args args) throws org.apache.thrift.TException {
+        splitTablet_result result = new splitTablet_result();
+        try {
+          iface.splitTablet(args.tinfo, args.credentials, args.extent, args.splitPoint);
+        } catch (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) {
+          result.sec = sec;
+        } catch (org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException nste) {
+          result.nste = nste;
+        }
+        return result;
+      }
+    }
+
+    public static class flushTablet<I extends Iface> extends org.apache.thrift.ProcessFunction<I, flushTablet_args> {
+      public flushTablet() {
+        super("flushTablet");
+      }
+
+      @Override
+      public flushTablet_args getEmptyArgsInstance() {
+        return new flushTablet_args();
+      }
+
+      @Override
+      protected boolean isOneway() {
+        return true;
+      }
+
+      @Override
+      protected boolean rethrowUnhandledExceptions() {
+        return false;
+      }
+
+      @Override
+      public org.apache.thrift.TBase getResult(I iface, flushTablet_args args) throws org.apache.thrift.TException {
+        iface.flushTablet(args.tinfo, args.credentials, args.lock, args.extent);
+        return null;
+      }
+    }
+
+  }
+
+  public static class AsyncProcessor<I extends AsyncIface> extends org.apache.thrift.TBaseAsyncProcessor<I> {
+    private static final org.slf4j.Logger _LOGGER = org.slf4j.LoggerFactory.getLogger(AsyncProcessor.class.getName());
+    public AsyncProcessor(I iface) {
+      super(iface, getProcessMap(new java.util.HashMap<java.lang.String, org.apache.thrift.AsyncProcessFunction<I, ? extends org.apache.thrift.TBase, ?>>()));
+    }
+
+    protected AsyncProcessor(I iface, java.util.Map<java.lang.String,  org.apache.thrift.AsyncProcessFunction<I, ? extends  org.apache.thrift.TBase, ?>> processMap) {
+      super(iface, getProcessMap(processMap));
+    }
+
+    private static <I extends AsyncIface> java.util.Map<java.lang.String,  org.apache.thrift.AsyncProcessFunction<I, ? extends  org.apache.thrift.TBase,?>> getProcessMap(java.util.Map<java.lang.String,  org.apache.thrift.AsyncProcessFunction<I, ? extends  org.apache.thrift.TBase, ?>> processMap) {
+      processMap.put("loadTablet", new loadTablet());
+      processMap.put("unloadTablet", new unloadTablet());
+      processMap.put("splitTablet", new splitTablet());
+      processMap.put("flushTablet", new flushTablet());
+      return processMap;
+    }
+
+    public static class loadTablet<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, loadTablet_args, Void> {
+      public loadTablet() {
+        super("loadTablet");
+      }
+
+      @Override
+      public loadTablet_args getEmptyArgsInstance() {
+        return new loadTablet_args();
+      }
+
+      @Override
+      public org.apache.thrift.async.AsyncMethodCallback<Void> 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<Void>() { 
+          @Override
+          public void onComplete(Void o) {
+          }
+          @Override
+          public void onError(java.lang.Exception e) {
+            if (e instanceof org.apache.thrift.transport.TTransportException) {
+              _LOGGER.error("TTransportException inside handler", e);
+              fb.close();
+            } else {
+              _LOGGER.error("Exception inside oneway handler", e);
+            }
+          }
+        };
+      }
+
+      @Override
+      protected boolean isOneway() {
+        return true;
+      }
+
+      @Override
+      public void start(I iface, loadTablet_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.loadTablet(args.tinfo, args.credentials, args.lock, args.extent,resultHandler);
+      }
+    }
+
+    public static class unloadTablet<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, unloadTablet_args, Void> {
+      public unloadTablet() {
+        super("unloadTablet");
+      }
+
+      @Override
+      public unloadTablet_args getEmptyArgsInstance() {
+        return new unloadTablet_args();
+      }
+
+      @Override
+      public org.apache.thrift.async.AsyncMethodCallback<Void> 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<Void>() { 
+          @Override
+          public void onComplete(Void o) {
+          }
+          @Override
+          public void onError(java.lang.Exception e) {
+            if (e instanceof org.apache.thrift.transport.TTransportException) {
+              _LOGGER.error("TTransportException inside handler", e);
+              fb.close();
+            } else {
+              _LOGGER.error("Exception inside oneway handler", e);
+            }
+          }
+        };
+      }
+
+      @Override
+      protected boolean isOneway() {
+        return true;
+      }
+
+      @Override
+      public void start(I iface, unloadTablet_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.unloadTablet(args.tinfo, args.credentials, args.lock, args.extent, args.goal, args.requestTime,resultHandler);
+      }
+    }
+
+    public static class splitTablet<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, splitTablet_args, Void> {
+      public splitTablet() {
+        super("splitTablet");
+      }
+
+      @Override
+      public splitTablet_args getEmptyArgsInstance() {
+        return new splitTablet_args();
+      }
+
+      @Override
+      public org.apache.thrift.async.AsyncMethodCallback<Void> 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<Void>() { 
+          @Override
+          public void onComplete(Void o) {
+            splitTablet_result result = new splitTablet_result();
+            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);
+            }
+          }
+          @Override
+          public void onError(java.lang.Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TSerializable msg;
+            splitTablet_result result = new splitTablet_result();
+            if (e instanceof org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) {
+              result.sec = (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) e;
+              result.setSecIsSet(true);
+              msg = result;
+            } else if (e instanceof org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException) {
+              result.nste = (org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException) e;
+              result.setNsteIsSet(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();
+            }
+          }
+        };
+      }
+
+      @Override
+      protected boolean isOneway() {
+        return false;
+      }
+
+      @Override
+      public void start(I iface, splitTablet_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.splitTablet(args.tinfo, args.credentials, args.extent, args.splitPoint,resultHandler);
+      }
+    }
+
+    public static class flushTablet<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, flushTablet_args, Void> {
+      public flushTablet() {
+        super("flushTablet");
+      }
+
+      @Override
+      public flushTablet_args getEmptyArgsInstance() {
+        return new flushTablet_args();
+      }
+
+      @Override
+      public org.apache.thrift.async.AsyncMethodCallback<Void> 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<Void>() { 
+          @Override
+          public void onComplete(Void o) {
+          }
+          @Override
+          public void onError(java.lang.Exception e) {
+            if (e instanceof org.apache.thrift.transport.TTransportException) {
+              _LOGGER.error("TTransportException inside handler", e);
+              fb.close();
+            } else {
+              _LOGGER.error("Exception inside oneway handler", e);
+            }
+          }
+        };
+      }
+
+      @Override
+      protected boolean isOneway() {
+        return true;
+      }
+
+      @Override
+      public void start(I iface, flushTablet_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.flushTablet(args.tinfo, args.credentials, args.lock, args.extent,resultHandler);
+      }
+    }
+
+  }
+
+  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+  public static class loadTablet_args implements org.apache.thrift.TBase<loadTablet_args, loadTablet_args._Fields>, java.io.Serializable, Cloneable, Comparable<loadTablet_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("loadTablet_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)5);
+    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)1);
+    private static final org.apache.thrift.protocol.TField LOCK_FIELD_DESC = new org.apache.thrift.protocol.TField("lock", org.apache.thrift.protocol.TType.STRING, (short)4);
+    private static final org.apache.thrift.protocol.TField EXTENT_FIELD_DESC = new org.apache.thrift.protocol.TField("extent", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new loadTablet_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new loadTablet_argsTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.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 lock; // required
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent; // 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)5, "tinfo"),
+      CREDENTIALS((short)1, "credentials"),
+      LOCK((short)4, "lock"),
+      EXTENT((short)2, "extent");
+
+      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 5: // TINFO
+            return TINFO;
+          case 1: // CREDENTIALS
+            return CREDENTIALS;
+          case 4: // LOCK
+            return LOCK;
+          case 2: // EXTENT
+            return EXTENT;
+          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;
+      }
+
+      @Override
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      @Override
+      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.clientImpl.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.LOCK, new org.apache.thrift.meta_data.FieldMetaData("lock", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      tmpMap.put(_Fields.EXTENT, new org.apache.thrift.meta_data.FieldMetaData("extent", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent.class)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(loadTablet_args.class, metaDataMap);
+    }
+
+    public loadTablet_args() {
+    }
+
+    public loadTablet_args(
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
+      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
+      java.lang.String lock,
+      org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent)
+    {
+      this();
+      this.tinfo = tinfo;
+      this.credentials = credentials;
+      this.lock = lock;
+      this.extent = extent;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public loadTablet_args(loadTablet_args other) {
+      if (other.isSetTinfo()) {
+        this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
+      }
+      if (other.isSetCredentials()) {
+        this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
+      }
+      if (other.isSetLock()) {
+        this.lock = other.lock;
+      }
+      if (other.isSetExtent()) {
+        this.extent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(other.extent);
+      }
+    }
+
+    @Override
+    public loadTablet_args deepCopy() {
+      return new loadTablet_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.tinfo = null;
+      this.credentials = null;
+      this.lock = null;
+      this.extent = null;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
+      return this.tinfo;
+    }
+
+    public loadTablet_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.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 loadTablet_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 getLock() {
+      return this.lock;
+    }
+
+    public loadTablet_args setLock(@org.apache.thrift.annotation.Nullable java.lang.String lock) {
+      this.lock = lock;
+      return this;
+    }
+
+    public void unsetLock() {
+      this.lock = null;
+    }
+
+    /** Returns true if field lock is set (has been assigned a value) and false otherwise */
+    public boolean isSetLock() {
+      return this.lock != null;
+    }
+
+    public void setLockIsSet(boolean value) {
+      if (!value) {
+        this.lock = null;
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.dataImpl.thrift.TKeyExtent getExtent() {
+      return this.extent;
+    }
+
+    public loadTablet_args setExtent(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent) {
+      this.extent = extent;
+      return this;
+    }
+
+    public void unsetExtent() {
+      this.extent = null;
+    }
+
+    /** Returns true if field extent is set (has been assigned a value) and false otherwise */
+    public boolean isSetExtent() {
+      return this.extent != null;
+    }
+
+    public void setExtentIsSet(boolean value) {
+      if (!value) {
+        this.extent = null;
+      }
+    }
+
+    @Override
+    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.clientImpl.thrift.TInfo)value);
+        }
+        break;
+
+      case CREDENTIALS:
+        if (value == null) {
+          unsetCredentials();
+        } else {
+          setCredentials((org.apache.accumulo.core.securityImpl.thrift.TCredentials)value);
+        }
+        break;
+
+      case LOCK:
+        if (value == null) {
+          unsetLock();
+        } else {
+          setLock((java.lang.String)value);
+        }
+        break;
+
+      case EXTENT:
+        if (value == null) {
+          unsetExtent();
+        } else {
+          setExtent((org.apache.accumulo.core.dataImpl.thrift.TKeyExtent)value);
+        }
+        break;
+
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    @Override
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case TINFO:
+        return getTinfo();
+
+      case CREDENTIALS:
+        return getCredentials();
+
+      case LOCK:
+        return getLock();
+
+      case EXTENT:
+        return getExtent();
+
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    @Override
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case TINFO:
+        return isSetTinfo();
+      case CREDENTIALS:
+        return isSetCredentials();
+      case LOCK:
+        return isSetLock();
+      case EXTENT:
+        return isSetExtent();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that instanceof loadTablet_args)
+        return this.equals((loadTablet_args)that);
+      return false;
+    }
+
+    public boolean equals(loadTablet_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_lock = true && this.isSetLock();
+      boolean that_present_lock = true && that.isSetLock();
+      if (this_present_lock || that_present_lock) {
+        if (!(this_present_lock && that_present_lock))
+          return false;
+        if (!this.lock.equals(that.lock))
+          return false;
+      }
+
+      boolean this_present_extent = true && this.isSetExtent();
+      boolean that_present_extent = true && that.isSetExtent();
+      if (this_present_extent || that_present_extent) {
+        if (!(this_present_extent && that_present_extent))
+          return false;
+        if (!this.extent.equals(that.extent))
+          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 + ((isSetLock()) ? 131071 : 524287);
+      if (isSetLock())
+        hashCode = hashCode * 8191 + lock.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetExtent()) ? 131071 : 524287);
+      if (isSetExtent())
+        hashCode = hashCode * 8191 + extent.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(loadTablet_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(isSetLock(), other.isSetLock());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetLock()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.lock, other.lock);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.compare(isSetExtent(), other.isSetExtent());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetExtent()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.extent, other.extent);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    @Override
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      scheme(iprot).read(iprot, this);
+    }
+
+    @Override
+    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("loadTablet_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("lock:");
+      if (this.lock == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.lock);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("extent:");
+      if (this.extent == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.extent);
+      }
+      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();
+      }
+      if (extent != null) {
+        extent.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 loadTablet_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public loadTablet_argsStandardScheme getScheme() {
+        return new loadTablet_argsStandardScheme();
+      }
+    }
+
+    private static class loadTablet_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<loadTablet_args> {
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol iprot, loadTablet_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 5: // TINFO
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
+                struct.tinfo.read(iprot);
+                struct.setTinfoIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 1: // 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 4: // LOCK
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.lock = iprot.readString();
+                struct.setLockIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 2: // EXTENT
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.extent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
+                struct.extent.read(iprot);
+                struct.setExtentIsSet(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();
+      }
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol oprot, loadTablet_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.credentials != null) {
+          oprot.writeFieldBegin(CREDENTIALS_FIELD_DESC);
+          struct.credentials.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.extent != null) {
+          oprot.writeFieldBegin(EXTENT_FIELD_DESC);
+          struct.extent.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.lock != null) {
+          oprot.writeFieldBegin(LOCK_FIELD_DESC);
+          oprot.writeString(struct.lock);
+          oprot.writeFieldEnd();
+        }
+        if (struct.tinfo != null) {
+          oprot.writeFieldBegin(TINFO_FIELD_DESC);
+          struct.tinfo.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class loadTablet_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public loadTablet_argsTupleScheme getScheme() {
+        return new loadTablet_argsTupleScheme();
+      }
+    }
+
+    private static class loadTablet_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<loadTablet_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, loadTablet_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.isSetLock()) {
+          optionals.set(2);
+        }
+        if (struct.isSetExtent()) {
+          optionals.set(3);
+        }
+        oprot.writeBitSet(optionals, 4);
+        if (struct.isSetTinfo()) {
+          struct.tinfo.write(oprot);
+        }
+        if (struct.isSetCredentials()) {
+          struct.credentials.write(oprot);
+        }
+        if (struct.isSetLock()) {
+          oprot.writeString(struct.lock);
+        }
+        if (struct.isSetExtent()) {
+          struct.extent.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, loadTablet_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(4);
+        if (incoming.get(0)) {
+          struct.tinfo = new org.apache.accumulo.core.clientImpl.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.lock = iprot.readString();
+          struct.setLockIsSet(true);
+        }
+        if (incoming.get(3)) {
+          struct.extent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
+          struct.extent.read(iprot);
+          struct.setExtentIsSet(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();
+    }
+  }
+
+  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+  public static class unloadTablet_args implements org.apache.thrift.TBase<unloadTablet_args, unloadTablet_args._Fields>, java.io.Serializable, Cloneable, Comparable<unloadTablet_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("unloadTablet_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)5);
+    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)1);
+    private static final org.apache.thrift.protocol.TField LOCK_FIELD_DESC = new org.apache.thrift.protocol.TField("lock", org.apache.thrift.protocol.TType.STRING, (short)4);
+    private static final org.apache.thrift.protocol.TField EXTENT_FIELD_DESC = new org.apache.thrift.protocol.TField("extent", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+    private static final org.apache.thrift.protocol.TField GOAL_FIELD_DESC = new org.apache.thrift.protocol.TField("goal", org.apache.thrift.protocol.TType.I32, (short)6);
+    private static final org.apache.thrift.protocol.TField REQUEST_TIME_FIELD_DESC = new org.apache.thrift.protocol.TField("requestTime", org.apache.thrift.protocol.TType.I64, (short)7);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new unloadTablet_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new unloadTablet_argsTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.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 lock; // required
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent; // required
+    /**
+     * 
+     * @see TUnloadTabletGoal
+     */
+    public @org.apache.thrift.annotation.Nullable TUnloadTabletGoal goal; // required
+    public long requestTime; // 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)5, "tinfo"),
+      CREDENTIALS((short)1, "credentials"),
+      LOCK((short)4, "lock"),
+      EXTENT((short)2, "extent"),
+      /**
+       * 
+       * @see TUnloadTabletGoal
+       */
+      GOAL((short)6, "goal"),
+      REQUEST_TIME((short)7, "requestTime");
+
+      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 5: // TINFO
+            return TINFO;
+          case 1: // CREDENTIALS
+            return CREDENTIALS;
+          case 4: // LOCK
+            return LOCK;
+          case 2: // EXTENT
+            return EXTENT;
+          case 6: // GOAL
+            return GOAL;
+          case 7: // REQUEST_TIME
+            return REQUEST_TIME;
+          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;
+      }
+
+      @Override
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      @Override
+      public java.lang.String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    private static final int __REQUESTTIME_ISSET_ID = 0;
+    private byte __isset_bitfield = 0;
+    public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.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.clientImpl.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.LOCK, new org.apache.thrift.meta_data.FieldMetaData("lock", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      tmpMap.put(_Fields.EXTENT, new org.apache.thrift.meta_data.FieldMetaData("extent", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent.class)));
+      tmpMap.put(_Fields.GOAL, new org.apache.thrift.meta_data.FieldMetaData("goal", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TUnloadTabletGoal.class)));
+      tmpMap.put(_Fields.REQUEST_TIME, new org.apache.thrift.meta_data.FieldMetaData("requestTime", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(unloadTablet_args.class, metaDataMap);
+    }
+
+    public unloadTablet_args() {
+    }
+
+    public unloadTablet_args(
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
+      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
+      java.lang.String lock,
+      org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent,
+      TUnloadTabletGoal goal,
+      long requestTime)
+    {
+      this();
+      this.tinfo = tinfo;
+      this.credentials = credentials;
+      this.lock = lock;
+      this.extent = extent;
+      this.goal = goal;
+      this.requestTime = requestTime;
+      setRequestTimeIsSet(true);
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public unloadTablet_args(unloadTablet_args other) {
+      __isset_bitfield = other.__isset_bitfield;
+      if (other.isSetTinfo()) {
+        this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
+      }
+      if (other.isSetCredentials()) {
+        this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
+      }
+      if (other.isSetLock()) {
+        this.lock = other.lock;
+      }
+      if (other.isSetExtent()) {
+        this.extent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(other.extent);
+      }
+      if (other.isSetGoal()) {
+        this.goal = other.goal;
+      }
+      this.requestTime = other.requestTime;
+    }
+
+    @Override
+    public unloadTablet_args deepCopy() {
+      return new unloadTablet_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.tinfo = null;
+      this.credentials = null;
+      this.lock = null;
+      this.extent = null;
+      this.goal = null;
+      setRequestTimeIsSet(false);
+      this.requestTime = 0;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
+      return this.tinfo;
+    }
+
+    public unloadTablet_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.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 unloadTablet_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 getLock() {
+      return this.lock;
+    }
+
+    public unloadTablet_args setLock(@org.apache.thrift.annotation.Nullable java.lang.String lock) {
+      this.lock = lock;
+      return this;
+    }
+
+    public void unsetLock() {
+      this.lock = null;
+    }
+
+    /** Returns true if field lock is set (has been assigned a value) and false otherwise */
+    public boolean isSetLock() {
+      return this.lock != null;
+    }
+
+    public void setLockIsSet(boolean value) {
+      if (!value) {
+        this.lock = null;
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.dataImpl.thrift.TKeyExtent getExtent() {
+      return this.extent;
+    }
+
+    public unloadTablet_args setExtent(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent) {
+      this.extent = extent;
+      return this;
+    }
+
+    public void unsetExtent() {
+      this.extent = null;
+    }
+
+    /** Returns true if field extent is set (has been assigned a value) and false otherwise */
+    public boolean isSetExtent() {
+      return this.extent != null;
+    }
+
+    public void setExtentIsSet(boolean value) {
+      if (!value) {
+        this.extent = null;
+      }
+    }
+
+    /**
+     * 
+     * @see TUnloadTabletGoal
+     */
+    @org.apache.thrift.annotation.Nullable
+    public TUnloadTabletGoal getGoal() {
+      return this.goal;
+    }
+
+    /**
+     * 
+     * @see TUnloadTabletGoal
+     */
+    public unloadTablet_args setGoal(@org.apache.thrift.annotation.Nullable TUnloadTabletGoal goal) {
+      this.goal = goal;
+      return this;
+    }
+
+    public void unsetGoal() {
+      this.goal = null;
+    }
+
+    /** Returns true if field goal is set (has been assigned a value) and false otherwise */
+    public boolean isSetGoal() {
+      return this.goal != null;
+    }
+
+    public void setGoalIsSet(boolean value) {
+      if (!value) {
+        this.goal = null;
+      }
+    }
+
+    public long getRequestTime() {
+      return this.requestTime;
+    }
+
+    public unloadTablet_args setRequestTime(long requestTime) {
+      this.requestTime = requestTime;
+      setRequestTimeIsSet(true);
+      return this;
+    }
+
+    public void unsetRequestTime() {
+      __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __REQUESTTIME_ISSET_ID);
+    }
+
+    /** Returns true if field requestTime is set (has been assigned a value) and false otherwise */
+    public boolean isSetRequestTime() {
+      return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __REQUESTTIME_ISSET_ID);
+    }
+
+    public void setRequestTimeIsSet(boolean value) {
+      __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __REQUESTTIME_ISSET_ID, value);
+    }
+
+    @Override
+    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.clientImpl.thrift.TInfo)value);
+        }
+        break;
+
+      case CREDENTIALS:
+        if (value == null) {
+          unsetCredentials();
+        } else {
+          setCredentials((org.apache.accumulo.core.securityImpl.thrift.TCredentials)value);
+        }
+        break;
+
+      case LOCK:
+        if (value == null) {
+          unsetLock();
+        } else {
+          setLock((java.lang.String)value);
+        }
+        break;
+
+      case EXTENT:
+        if (value == null) {
+          unsetExtent();
+        } else {
+          setExtent((org.apache.accumulo.core.dataImpl.thrift.TKeyExtent)value);
+        }
+        break;
+
+      case GOAL:
+        if (value == null) {
+          unsetGoal();
+        } else {
+          setGoal((TUnloadTabletGoal)value);
+        }
+        break;
+
+      case REQUEST_TIME:
+        if (value == null) {
+          unsetRequestTime();
+        } else {
+          setRequestTime((java.lang.Long)value);
+        }
+        break;
+
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    @Override
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case TINFO:
+        return getTinfo();
+
+      case CREDENTIALS:
+        return getCredentials();
+
+      case LOCK:
+        return getLock();
+
+      case EXTENT:
+        return getExtent();
+
+      case GOAL:
+        return getGoal();
+
+      case REQUEST_TIME:
+        return getRequestTime();
+
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    @Override
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case TINFO:
+        return isSetTinfo();
+      case CREDENTIALS:
+        return isSetCredentials();
+      case LOCK:
+        return isSetLock();
+      case EXTENT:
+        return isSetExtent();
+      case GOAL:
+        return isSetGoal();
+      case REQUEST_TIME:
+        return isSetRequestTime();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that instanceof unloadTablet_args)
+        return this.equals((unloadTablet_args)that);
+      return false;
+    }
+
+    public boolean equals(unloadTablet_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_lock = true && this.isSetLock();
+      boolean that_present_lock = true && that.isSetLock();
+      if (this_present_lock || that_present_lock) {
+        if (!(this_present_lock && that_present_lock))
+          return false;
+        if (!this.lock.equals(that.lock))
+          return false;
+      }
+
+      boolean this_present_extent = true && this.isSetExtent();
+      boolean that_present_extent = true && that.isSetExtent();
+      if (this_present_extent || that_present_extent) {
+        if (!(this_present_extent && that_present_extent))
+          return false;
+        if (!this.extent.equals(that.extent))
+          return false;
+      }
+
+      boolean this_present_goal = true && this.isSetGoal();
+      boolean that_present_goal = true && that.isSetGoal();
+      if (this_present_goal || that_present_goal) {
+        if (!(this_present_goal && that_present_goal))
+          return false;
+        if (!this.goal.equals(that.goal))
+          return false;
+      }
+
+      boolean this_present_requestTime = true;
+      boolean that_present_requestTime = true;
+      if (this_present_requestTime || that_present_requestTime) {
+        if (!(this_present_requestTime && that_present_requestTime))
+          return false;
+        if (this.requestTime != that.requestTime)
+          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 + ((isSetLock()) ? 131071 : 524287);
+      if (isSetLock())
+        hashCode = hashCode * 8191 + lock.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetExtent()) ? 131071 : 524287);
+      if (isSetExtent())
+        hashCode = hashCode * 8191 + extent.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetGoal()) ? 131071 : 524287);
+      if (isSetGoal())
+        hashCode = hashCode * 8191 + goal.getValue();
+
+      hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(requestTime);
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(unloadTablet_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(isSetLock(), other.isSetLock());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetLock()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.lock, other.lock);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.compare(isSetExtent(), other.isSetExtent());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetExtent()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.extent, other.extent);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.compare(isSetGoal(), other.isSetGoal());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetGoal()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.goal, other.goal);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.compare(isSetRequestTime(), other.isSetRequestTime());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetRequestTime()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.requestTime, other.requestTime);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    @Override
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      scheme(iprot).read(iprot, this);
+    }
+
+    @Override
+    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("unloadTablet_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("lock:");
+      if (this.lock == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.lock);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("extent:");
+      if (this.extent == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.extent);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("goal:");
+      if (this.goal == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.goal);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("requestTime:");
+      sb.append(this.requestTime);
+      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();
+      }
+      if (extent != null) {
+        extent.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 {
+        // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+        __isset_bitfield = 0;
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class unloadTablet_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public unloadTablet_argsStandardScheme getScheme() {
+        return new unloadTablet_argsStandardScheme();
+      }
+    }
+
+    private static class unloadTablet_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<unloadTablet_args> {
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol iprot, unloadTablet_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 5: // TINFO
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
+                struct.tinfo.read(iprot);
+                struct.setTinfoIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 1: // 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 4: // LOCK
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.lock = iprot.readString();
+                struct.setLockIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 2: // EXTENT
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.extent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
+                struct.extent.read(iprot);
+                struct.setExtentIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 6: // GOAL
+              if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+                struct.goal = org.apache.accumulo.core.tablet.thrift.TUnloadTabletGoal.findByValue(iprot.readI32());
+                struct.setGoalIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 7: // REQUEST_TIME
+              if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+                struct.requestTime = iprot.readI64();
+                struct.setRequestTimeIsSet(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();
+      }
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol oprot, unloadTablet_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.credentials != null) {
+          oprot.writeFieldBegin(CREDENTIALS_FIELD_DESC);
+          struct.credentials.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.extent != null) {
+          oprot.writeFieldBegin(EXTENT_FIELD_DESC);
+          struct.extent.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.lock != null) {
+          oprot.writeFieldBegin(LOCK_FIELD_DESC);
+          oprot.writeString(struct.lock);
+          oprot.writeFieldEnd();
+        }
+        if (struct.tinfo != null) {
+          oprot.writeFieldBegin(TINFO_FIELD_DESC);
+          struct.tinfo.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.goal != null) {
+          oprot.writeFieldBegin(GOAL_FIELD_DESC);
+          oprot.writeI32(struct.goal.getValue());
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldBegin(REQUEST_TIME_FIELD_DESC);
+        oprot.writeI64(struct.requestTime);
+        oprot.writeFieldEnd();
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class unloadTablet_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public unloadTablet_argsTupleScheme getScheme() {
+        return new unloadTablet_argsTupleScheme();
+      }
+    }
+
+    private static class unloadTablet_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<unloadTablet_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, unloadTablet_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.isSetLock()) {
+          optionals.set(2);
+        }
+        if (struct.isSetExtent()) {
+          optionals.set(3);
+        }
+        if (struct.isSetGoal()) {
+          optionals.set(4);
+        }
+        if (struct.isSetRequestTime()) {
+          optionals.set(5);
+        }
+        oprot.writeBitSet(optionals, 6);
+        if (struct.isSetTinfo()) {
+          struct.tinfo.write(oprot);
+        }
+        if (struct.isSetCredentials()) {
+          struct.credentials.write(oprot);
+        }
+        if (struct.isSetLock()) {
+          oprot.writeString(struct.lock);
+        }
+        if (struct.isSetExtent()) {
+          struct.extent.write(oprot);
+        }
+        if (struct.isSetGoal()) {
+          oprot.writeI32(struct.goal.getValue());
+        }
+        if (struct.isSetRequestTime()) {
+          oprot.writeI64(struct.requestTime);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, unloadTablet_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(6);
+        if (incoming.get(0)) {
+          struct.tinfo = new org.apache.accumulo.core.clientImpl.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.lock = iprot.readString();
+          struct.setLockIsSet(true);
+        }
+        if (incoming.get(3)) {
+          struct.extent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
+          struct.extent.read(iprot);
+          struct.setExtentIsSet(true);
+        }
+        if (incoming.get(4)) {
+          struct.goal = org.apache.accumulo.core.tablet.thrift.TUnloadTabletGoal.findByValue(iprot.readI32());
+          struct.setGoalIsSet(true);
+        }
+        if (incoming.get(5)) {
+          struct.requestTime = iprot.readI64();
+          struct.setRequestTimeIsSet(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();
+    }
+  }
+
+  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+  public static class splitTablet_args implements org.apache.thrift.TBase<splitTablet_args, splitTablet_args._Fields>, java.io.Serializable, Cloneable, Comparable<splitTablet_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("splitTablet_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)4);
+    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)1);
+    private static final org.apache.thrift.protocol.TField EXTENT_FIELD_DESC = new org.apache.thrift.protocol.TField("extent", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+    private static final org.apache.thrift.protocol.TField SPLIT_POINT_FIELD_DESC = new org.apache.thrift.protocol.TField("splitPoint", org.apache.thrift.protocol.TType.STRING, (short)3);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new splitTablet_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new splitTablet_argsTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.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 org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent; // required
+    public @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer splitPoint; // 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)4, "tinfo"),
+      CREDENTIALS((short)1, "credentials"),
+      EXTENT((short)2, "extent"),
+      SPLIT_POINT((short)3, "splitPoint");
+
+      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 4: // TINFO
+            return TINFO;
+          case 1: // CREDENTIALS
+            return CREDENTIALS;
+          case 2: // EXTENT
+            return EXTENT;
+          case 3: // SPLIT_POINT
+            return SPLIT_POINT;
+          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;
+      }
+
+      @Override
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      @Override
+      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.clientImpl.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.EXTENT, new org.apache.thrift.meta_data.FieldMetaData("extent", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent.class)));
+      tmpMap.put(_Fields.SPLIT_POINT, new org.apache.thrift.meta_data.FieldMetaData("splitPoint", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING          , true)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(splitTablet_args.class, metaDataMap);
+    }
+
+    public splitTablet_args() {
+    }
+
+    public splitTablet_args(
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
+      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
+      org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent,
+      java.nio.ByteBuffer splitPoint)
+    {
+      this();
+      this.tinfo = tinfo;
+      this.credentials = credentials;
+      this.extent = extent;
+      this.splitPoint = org.apache.thrift.TBaseHelper.copyBinary(splitPoint);
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public splitTablet_args(splitTablet_args other) {
+      if (other.isSetTinfo()) {
+        this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
+      }
+      if (other.isSetCredentials()) {
+        this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
+      }
+      if (other.isSetExtent()) {
+        this.extent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(other.extent);
+      }
+      if (other.isSetSplitPoint()) {
+        this.splitPoint = org.apache.thrift.TBaseHelper.copyBinary(other.splitPoint);
+      }
+    }
+
+    @Override
+    public splitTablet_args deepCopy() {
+      return new splitTablet_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.tinfo = null;
+      this.credentials = null;
+      this.extent = null;
+      this.splitPoint = null;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
+      return this.tinfo;
+    }
+
+    public splitTablet_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.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 splitTablet_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 org.apache.accumulo.core.dataImpl.thrift.TKeyExtent getExtent() {
+      return this.extent;
+    }
+
+    public splitTablet_args setExtent(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent) {
+      this.extent = extent;
+      return this;
+    }
+
+    public void unsetExtent() {
+      this.extent = null;
+    }
+
+    /** Returns true if field extent is set (has been assigned a value) and false otherwise */
+    public boolean isSetExtent() {
+      return this.extent != null;
+    }
+
+    public void setExtentIsSet(boolean value) {
+      if (!value) {
+        this.extent = null;
+      }
+    }
+
+    public byte[] getSplitPoint() {
+      setSplitPoint(org.apache.thrift.TBaseHelper.rightSize(splitPoint));
+      return splitPoint == null ? null : splitPoint.array();
+    }
+
+    public java.nio.ByteBuffer bufferForSplitPoint() {
+      return org.apache.thrift.TBaseHelper.copyBinary(splitPoint);
+    }
+
+    public splitTablet_args setSplitPoint(byte[] splitPoint) {
+      this.splitPoint = splitPoint == null ? (java.nio.ByteBuffer)null     : java.nio.ByteBuffer.wrap(splitPoint.clone());
+      return this;
+    }
+
+    public splitTablet_args setSplitPoint(@org.apache.thrift.annotation.Nullable java.nio.ByteBuffer splitPoint) {
+      this.splitPoint = org.apache.thrift.TBaseHelper.copyBinary(splitPoint);
+      return this;
+    }
+
+    public void unsetSplitPoint() {
+      this.splitPoint = null;
+    }
+
+    /** Returns true if field splitPoint is set (has been assigned a value) and false otherwise */
+    public boolean isSetSplitPoint() {
+      return this.splitPoint != null;
+    }
+
+    public void setSplitPointIsSet(boolean value) {
+      if (!value) {
+        this.splitPoint = null;
+      }
+    }
+
+    @Override
+    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.clientImpl.thrift.TInfo)value);
+        }
+        break;
+
+      case CREDENTIALS:
+        if (value == null) {
+          unsetCredentials();
+        } else {
+          setCredentials((org.apache.accumulo.core.securityImpl.thrift.TCredentials)value);
+        }
+        break;
+
+      case EXTENT:
+        if (value == null) {
+          unsetExtent();
+        } else {
+          setExtent((org.apache.accumulo.core.dataImpl.thrift.TKeyExtent)value);
+        }
+        break;
+
+      case SPLIT_POINT:
+        if (value == null) {
+          unsetSplitPoint();
+        } else {
+          if (value instanceof byte[]) {
+            setSplitPoint((byte[])value);
+          } else {
+            setSplitPoint((java.nio.ByteBuffer)value);
+          }
+        }
+        break;
+
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    @Override
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case TINFO:
+        return getTinfo();
+
+      case CREDENTIALS:
+        return getCredentials();
+
+      case EXTENT:
+        return getExtent();
+
+      case SPLIT_POINT:
+        return getSplitPoint();
+
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    @Override
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case TINFO:
+        return isSetTinfo();
+      case CREDENTIALS:
+        return isSetCredentials();
+      case EXTENT:
+        return isSetExtent();
+      case SPLIT_POINT:
+        return isSetSplitPoint();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that instanceof splitTablet_args)
+        return this.equals((splitTablet_args)that);
+      return false;
+    }
+
+    public boolean equals(splitTablet_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_extent = true && this.isSetExtent();
+      boolean that_present_extent = true && that.isSetExtent();
+      if (this_present_extent || that_present_extent) {
+        if (!(this_present_extent && that_present_extent))
+          return false;
+        if (!this.extent.equals(that.extent))
+          return false;
+      }
+
+      boolean this_present_splitPoint = true && this.isSetSplitPoint();
+      boolean that_present_splitPoint = true && that.isSetSplitPoint();
+      if (this_present_splitPoint || that_present_splitPoint) {
+        if (!(this_present_splitPoint && that_present_splitPoint))
+          return false;
+        if (!this.splitPoint.equals(that.splitPoint))
+          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 + ((isSetExtent()) ? 131071 : 524287);
+      if (isSetExtent())
+        hashCode = hashCode * 8191 + extent.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetSplitPoint()) ? 131071 : 524287);
+      if (isSetSplitPoint())
+        hashCode = hashCode * 8191 + splitPoint.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(splitTablet_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(isSetExtent(), other.isSetExtent());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetExtent()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.extent, other.extent);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.compare(isSetSplitPoint(), other.isSetSplitPoint());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetSplitPoint()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.splitPoint, other.splitPoint);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    @Override
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      scheme(iprot).read(iprot, this);
+    }
+
+    @Override
+    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("splitTablet_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("extent:");
+      if (this.extent == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.extent);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("splitPoint:");
+      if (this.splitPoint == null) {
+        sb.append("null");
+      } else {
+        org.apache.thrift.TBaseHelper.toString(this.splitPoint, sb);
+      }
+      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();
+      }
+      if (extent != null) {
+        extent.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 splitTablet_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public splitTablet_argsStandardScheme getScheme() {
+        return new splitTablet_argsStandardScheme();
+      }
+    }
+
+    private static class splitTablet_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<splitTablet_args> {
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol iprot, splitTablet_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 4: // TINFO
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
+                struct.tinfo.read(iprot);
+                struct.setTinfoIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 1: // 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: // EXTENT
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.extent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
+                struct.extent.read(iprot);
+                struct.setExtentIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 3: // SPLIT_POINT
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.splitPoint = iprot.readBinary();
+                struct.setSplitPointIsSet(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();
+      }
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol oprot, splitTablet_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.credentials != null) {
+          oprot.writeFieldBegin(CREDENTIALS_FIELD_DESC);
+          struct.credentials.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.extent != null) {
+          oprot.writeFieldBegin(EXTENT_FIELD_DESC);
+          struct.extent.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.splitPoint != null) {
+          oprot.writeFieldBegin(SPLIT_POINT_FIELD_DESC);
+          oprot.writeBinary(struct.splitPoint);
+          oprot.writeFieldEnd();
+        }
+        if (struct.tinfo != null) {
+          oprot.writeFieldBegin(TINFO_FIELD_DESC);
+          struct.tinfo.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class splitTablet_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public splitTablet_argsTupleScheme getScheme() {
+        return new splitTablet_argsTupleScheme();
+      }
+    }
+
+    private static class splitTablet_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<splitTablet_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, splitTablet_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.isSetExtent()) {
+          optionals.set(2);
+        }
+        if (struct.isSetSplitPoint()) {
+          optionals.set(3);
+        }
+        oprot.writeBitSet(optionals, 4);
+        if (struct.isSetTinfo()) {
+          struct.tinfo.write(oprot);
+        }
+        if (struct.isSetCredentials()) {
+          struct.credentials.write(oprot);
+        }
+        if (struct.isSetExtent()) {
+          struct.extent.write(oprot);
+        }
+        if (struct.isSetSplitPoint()) {
+          oprot.writeBinary(struct.splitPoint);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, splitTablet_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(4);
+        if (incoming.get(0)) {
+          struct.tinfo = new org.apache.accumulo.core.clientImpl.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.extent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
+          struct.extent.read(iprot);
+          struct.setExtentIsSet(true);
+        }
+        if (incoming.get(3)) {
+          struct.splitPoint = iprot.readBinary();
+          struct.setSplitPointIsSet(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();
+    }
+  }
+
+  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+  public static class splitTablet_result implements org.apache.thrift.TBase<splitTablet_result, splitTablet_result._Fields>, java.io.Serializable, Cloneable, Comparable<splitTablet_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("splitTablet_result");
+
+    private static final org.apache.thrift.protocol.TField SEC_FIELD_DESC = new org.apache.thrift.protocol.TField("sec", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+    private static final org.apache.thrift.protocol.TField NSTE_FIELD_DESC = new org.apache.thrift.protocol.TField("nste", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new splitTablet_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new splitTablet_resultTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec; // required
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException nste; // 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 {
+      SEC((short)1, "sec"),
+      NSTE((short)2, "nste");
+
+      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: // SEC
+            return SEC;
+          case 2: // NSTE
+            return NSTE;
+          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;
+      }
+
+      @Override
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      @Override
+      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.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, org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException.class)));
+      tmpMap.put(_Fields.NSTE, new org.apache.thrift.meta_data.FieldMetaData("nste", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException.class)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(splitTablet_result.class, metaDataMap);
+    }
+
+    public splitTablet_result() {
+    }
+
+    public splitTablet_result(
+      org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec,
+      org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException nste)
+    {
+      this();
+      this.sec = sec;
+      this.nste = nste;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public splitTablet_result(splitTablet_result other) {
+      if (other.isSetSec()) {
+        this.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException(other.sec);
+      }
+      if (other.isSetNste()) {
+        this.nste = new org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException(other.nste);
+      }
+    }
+
+    @Override
+    public splitTablet_result deepCopy() {
+      return new splitTablet_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.sec = null;
+      this.nste = null;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException getSec() {
+      return this.sec;
+    }
+
+    public splitTablet_result setSec(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.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 org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException getNste() {
+      return this.nste;
+    }
+
+    public splitTablet_result setNste(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException nste) {
+      this.nste = nste;
+      return this;
+    }
+
+    public void unsetNste() {
+      this.nste = null;
+    }
+
+    /** Returns true if field nste is set (has been assigned a value) and false otherwise */
+    public boolean isSetNste() {
+      return this.nste != null;
+    }
+
+    public void setNsteIsSet(boolean value) {
+      if (!value) {
+        this.nste = null;
+      }
+    }
+
+    @Override
+    public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+      switch (field) {
+      case SEC:
+        if (value == null) {
+          unsetSec();
+        } else {
+          setSec((org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException)value);
+        }
+        break;
+
+      case NSTE:
+        if (value == null) {
+          unsetNste();
+        } else {
+          setNste((org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException)value);
+        }
+        break;
+
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    @Override
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case SEC:
+        return getSec();
+
+      case NSTE:
+        return getNste();
+
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    @Override
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case SEC:
+        return isSetSec();
+      case NSTE:
+        return isSetNste();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that instanceof splitTablet_result)
+        return this.equals((splitTablet_result)that);
+      return false;
+    }
+
+    public boolean equals(splitTablet_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_nste = true && this.isSetNste();
+      boolean that_present_nste = true && that.isSetNste();
+      if (this_present_nste || that_present_nste) {
+        if (!(this_present_nste && that_present_nste))
+          return false;
+        if (!this.nste.equals(that.nste))
+          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 + ((isSetNste()) ? 131071 : 524287);
+      if (isSetNste())
+        hashCode = hashCode * 8191 + nste.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(splitTablet_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(isSetNste(), other.isSetNste());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetNste()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nste, other.nste);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    @Override
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    @Override
+    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("splitTablet_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("nste:");
+      if (this.nste == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.nste);
+      }
+      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 splitTablet_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public splitTablet_resultStandardScheme getScheme() {
+        return new splitTablet_resultStandardScheme();
+      }
+    }
+
+    private static class splitTablet_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<splitTablet_result> {
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol iprot, splitTablet_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 org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException();
+                struct.sec.read(iprot);
+                struct.setSecIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 2: // NSTE
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.nste = new org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException();
+                struct.nste.read(iprot);
+                struct.setNsteIsSet(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();
+      }
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol oprot, splitTablet_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.nste != null) {
+          oprot.writeFieldBegin(NSTE_FIELD_DESC);
+          struct.nste.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class splitTablet_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public splitTablet_resultTupleScheme getScheme() {
+        return new splitTablet_resultTupleScheme();
+      }
+    }
+
+    private static class splitTablet_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<splitTablet_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, splitTablet_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.isSetNste()) {
+          optionals.set(1);
+        }
+        oprot.writeBitSet(optionals, 2);
+        if (struct.isSetSec()) {
+          struct.sec.write(oprot);
+        }
+        if (struct.isSetNste()) {
+          struct.nste.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, splitTablet_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 org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException();
+          struct.sec.read(iprot);
+          struct.setSecIsSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.nste = new org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException();
+          struct.nste.read(iprot);
+          struct.setNsteIsSet(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();
+    }
+  }
+
+  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+  public static class flushTablet_args implements org.apache.thrift.TBase<flushTablet_args, flushTablet_args._Fields>, java.io.Serializable, Cloneable, Comparable<flushTablet_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("flushTablet_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.protocol.TField LOCK_FIELD_DESC = new org.apache.thrift.protocol.TField("lock", org.apache.thrift.protocol.TType.STRING, (short)3);
+    private static final org.apache.thrift.protocol.TField EXTENT_FIELD_DESC = new org.apache.thrift.protocol.TField("extent", org.apache.thrift.protocol.TType.STRUCT, (short)4);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new flushTablet_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new flushTablet_argsTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.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 lock; // required
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent; // 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"),
+      LOCK((short)3, "lock"),
+      EXTENT((short)4, "extent");
+
+      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;
+          case 3: // LOCK
+            return LOCK;
+          case 4: // EXTENT
+            return EXTENT;
+          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;
+      }
+
+      @Override
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      @Override
+      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.clientImpl.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.LOCK, new org.apache.thrift.meta_data.FieldMetaData("lock", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      tmpMap.put(_Fields.EXTENT, new org.apache.thrift.meta_data.FieldMetaData("extent", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent.class)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(flushTablet_args.class, metaDataMap);
+    }
+
+    public flushTablet_args() {
+    }
+
+    public flushTablet_args(
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
+      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
+      java.lang.String lock,
+      org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent)
+    {
+      this();
+      this.tinfo = tinfo;
+      this.credentials = credentials;
+      this.lock = lock;
+      this.extent = extent;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public flushTablet_args(flushTablet_args other) {
+      if (other.isSetTinfo()) {
+        this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
+      }
+      if (other.isSetCredentials()) {
+        this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
+      }
+      if (other.isSetLock()) {
+        this.lock = other.lock;
+      }
+      if (other.isSetExtent()) {
+        this.extent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(other.extent);
+      }
+    }
+
+    @Override
+    public flushTablet_args deepCopy() {
+      return new flushTablet_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.tinfo = null;
+      this.credentials = null;
+      this.lock = null;
+      this.extent = null;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
+      return this.tinfo;
+    }
+
+    public flushTablet_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.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 flushTablet_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 getLock() {
+      return this.lock;
+    }
+
+    public flushTablet_args setLock(@org.apache.thrift.annotation.Nullable java.lang.String lock) {
+      this.lock = lock;
+      return this;
+    }
+
+    public void unsetLock() {
+      this.lock = null;
+    }
+
+    /** Returns true if field lock is set (has been assigned a value) and false otherwise */
+    public boolean isSetLock() {
+      return this.lock != null;
+    }
+
+    public void setLockIsSet(boolean value) {
+      if (!value) {
+        this.lock = null;
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.dataImpl.thrift.TKeyExtent getExtent() {
+      return this.extent;
+    }
+
+    public flushTablet_args setExtent(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent) {
+      this.extent = extent;
+      return this;
+    }
+
+    public void unsetExtent() {
+      this.extent = null;
+    }
+
+    /** Returns true if field extent is set (has been assigned a value) and false otherwise */
+    public boolean isSetExtent() {
+      return this.extent != null;
+    }
+
+    public void setExtentIsSet(boolean value) {
+      if (!value) {
+        this.extent = null;
+      }
+    }
+
+    @Override
+    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.clientImpl.thrift.TInfo)value);
+        }
+        break;
+
+      case CREDENTIALS:
+        if (value == null) {
+          unsetCredentials();
+        } else {
+          setCredentials((org.apache.accumulo.core.securityImpl.thrift.TCredentials)value);
+        }
+        break;
+
+      case LOCK:
+        if (value == null) {
+          unsetLock();
+        } else {
+          setLock((java.lang.String)value);
+        }
+        break;
+
+      case EXTENT:
+        if (value == null) {
+          unsetExtent();
+        } else {
+          setExtent((org.apache.accumulo.core.dataImpl.thrift.TKeyExtent)value);
+        }
+        break;
+
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    @Override
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case TINFO:
+        return getTinfo();
+
+      case CREDENTIALS:
+        return getCredentials();
+
+      case LOCK:
+        return getLock();
+
+      case EXTENT:
+        return getExtent();
+
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    @Override
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case TINFO:
+        return isSetTinfo();
+      case CREDENTIALS:
+        return isSetCredentials();
+      case LOCK:
+        return isSetLock();
+      case EXTENT:
+        return isSetExtent();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that instanceof flushTablet_args)
+        return this.equals((flushTablet_args)that);
+      return false;
+    }
+
+    public boolean equals(flushTablet_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_lock = true && this.isSetLock();
+      boolean that_present_lock = true && that.isSetLock();
+      if (this_present_lock || that_present_lock) {
+        if (!(this_present_lock && that_present_lock))
+          return false;
+        if (!this.lock.equals(that.lock))
+          return false;
+      }
+
+      boolean this_present_extent = true && this.isSetExtent();
+      boolean that_present_extent = true && that.isSetExtent();
+      if (this_present_extent || that_present_extent) {
+        if (!(this_present_extent && that_present_extent))
+          return false;
+        if (!this.extent.equals(that.extent))
+          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 + ((isSetLock()) ? 131071 : 524287);
+      if (isSetLock())
+        hashCode = hashCode * 8191 + lock.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetExtent()) ? 131071 : 524287);
+      if (isSetExtent())
+        hashCode = hashCode * 8191 + extent.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(flushTablet_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(isSetLock(), other.isSetLock());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetLock()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.lock, other.lock);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.compare(isSetExtent(), other.isSetExtent());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetExtent()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.extent, other.extent);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    @Override
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      scheme(iprot).read(iprot, this);
+    }
+
+    @Override
+    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("flushTablet_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("lock:");
+      if (this.lock == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.lock);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("extent:");
+      if (this.extent == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.extent);
+      }
+      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();
+      }
+      if (extent != null) {
+        extent.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 flushTablet_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public flushTablet_argsStandardScheme getScheme() {
+        return new flushTablet_argsStandardScheme();
+      }
+    }
+
+    private static class flushTablet_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<flushTablet_args> {
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol iprot, flushTablet_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.clientImpl.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;
+            case 3: // LOCK
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.lock = iprot.readString();
+                struct.setLockIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 4: // EXTENT
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.extent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
+                struct.extent.read(iprot);
+                struct.setExtentIsSet(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();
+      }
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol oprot, flushTablet_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();
+        }
+        if (struct.lock != null) {
+          oprot.writeFieldBegin(LOCK_FIELD_DESC);
+          oprot.writeString(struct.lock);
+          oprot.writeFieldEnd();
+        }
+        if (struct.extent != null) {
+          oprot.writeFieldBegin(EXTENT_FIELD_DESC);
+          struct.extent.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class flushTablet_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public flushTablet_argsTupleScheme getScheme() {
+        return new flushTablet_argsTupleScheme();
+      }
+    }
+
+    private static class flushTablet_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<flushTablet_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, flushTablet_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.isSetLock()) {
+          optionals.set(2);
+        }
+        if (struct.isSetExtent()) {
+          optionals.set(3);
+        }
+        oprot.writeBitSet(optionals, 4);
+        if (struct.isSetTinfo()) {
+          struct.tinfo.write(oprot);
+        }
+        if (struct.isSetCredentials()) {
+          struct.credentials.write(oprot);
+        }
+        if (struct.isSetLock()) {
+          oprot.writeString(struct.lock);
+        }
+        if (struct.isSetExtent()) {
+          struct.extent.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, flushTablet_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(4);
+        if (incoming.get(0)) {
+          struct.tinfo = new org.apache.accumulo.core.clientImpl.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.lock = iprot.readString();
+          struct.setLockIsSet(true);
+        }
+        if (incoming.get(3)) {
+          struct.extent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
+          struct.extent.read(iprot);
+          struct.setExtentIsSet(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();
+    }
+  }
+
+  private static void unusedMethod() {}
+}
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/ConstraintViolationException.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletingest/thrift/ConstraintViolationException.java
similarity index 99%
rename from core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/ConstraintViolationException.java
rename to core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletingest/thrift/ConstraintViolationException.java
index ca15e65..41e5748 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/ConstraintViolationException.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletingest/thrift/ConstraintViolationException.java
@@ -22,7 +22,7 @@
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
  */
-package org.apache.accumulo.core.tabletserver.thrift;
+package org.apache.accumulo.core.tabletingest.thrift;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
 public class ConstraintViolationException extends org.apache.thrift.TException implements org.apache.thrift.TBase<ConstraintViolationException, ConstraintViolationException._Fields>, java.io.Serializable, Cloneable, Comparable<ConstraintViolationException> {
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/dataImpl/thrift/MapFileInfo.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletingest/thrift/DataFileInfo.java
similarity index 86%
rename from core/src/main/thrift-gen-java/org/apache/accumulo/core/dataImpl/thrift/MapFileInfo.java
rename to core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletingest/thrift/DataFileInfo.java
index 0d7de77..12276fd 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/dataImpl/thrift/MapFileInfo.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletingest/thrift/DataFileInfo.java
@@ -22,16 +22,16 @@
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
  */
-package org.apache.accumulo.core.dataImpl.thrift;
+package org.apache.accumulo.core.tabletingest.thrift;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-public class MapFileInfo implements org.apache.thrift.TBase<MapFileInfo, MapFileInfo._Fields>, java.io.Serializable, Cloneable, Comparable<MapFileInfo> {
-  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("MapFileInfo");
+public class DataFileInfo implements org.apache.thrift.TBase<DataFileInfo, DataFileInfo._Fields>, java.io.Serializable, Cloneable, Comparable<DataFileInfo> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("DataFileInfo");
 
   private static final org.apache.thrift.protocol.TField ESTIMATED_SIZE_FIELD_DESC = new org.apache.thrift.protocol.TField("estimatedSize", org.apache.thrift.protocol.TType.I64, (short)1);
 
-  private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new MapFileInfoStandardSchemeFactory();
-  private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new MapFileInfoTupleSchemeFactory();
+  private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new DataFileInfoStandardSchemeFactory();
+  private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new DataFileInfoTupleSchemeFactory();
 
   public long estimatedSize; // required
 
@@ -106,13 +106,13 @@
     tmpMap.put(_Fields.ESTIMATED_SIZE, new org.apache.thrift.meta_data.FieldMetaData("estimatedSize", org.apache.thrift.TFieldRequirementType.DEFAULT, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
     metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(MapFileInfo.class, metaDataMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(DataFileInfo.class, metaDataMap);
   }
 
-  public MapFileInfo() {
+  public DataFileInfo() {
   }
 
-  public MapFileInfo(
+  public DataFileInfo(
     long estimatedSize)
   {
     this();
@@ -123,14 +123,14 @@
   /**
    * Performs a deep copy on <i>other</i>.
    */
-  public MapFileInfo(MapFileInfo other) {
+  public DataFileInfo(DataFileInfo other) {
     __isset_bitfield = other.__isset_bitfield;
     this.estimatedSize = other.estimatedSize;
   }
 
   @Override
-  public MapFileInfo deepCopy() {
-    return new MapFileInfo(this);
+  public DataFileInfo deepCopy() {
+    return new DataFileInfo(this);
   }
 
   @Override
@@ -143,7 +143,7 @@
     return this.estimatedSize;
   }
 
-  public MapFileInfo setEstimatedSize(long estimatedSize) {
+  public DataFileInfo setEstimatedSize(long estimatedSize) {
     this.estimatedSize = estimatedSize;
     setEstimatedSizeIsSet(true);
     return this;
@@ -203,12 +203,12 @@
 
   @Override
   public boolean equals(java.lang.Object that) {
-    if (that instanceof MapFileInfo)
-      return this.equals((MapFileInfo)that);
+    if (that instanceof DataFileInfo)
+      return this.equals((DataFileInfo)that);
     return false;
   }
 
-  public boolean equals(MapFileInfo that) {
+  public boolean equals(DataFileInfo that) {
     if (that == null)
       return false;
     if (this == that)
@@ -236,7 +236,7 @@
   }
 
   @Override
-  public int compareTo(MapFileInfo other) {
+  public int compareTo(DataFileInfo other) {
     if (!getClass().equals(other.getClass())) {
       return getClass().getName().compareTo(other.getClass().getName());
     }
@@ -274,7 +274,7 @@
 
   @Override
   public java.lang.String toString() {
-    java.lang.StringBuilder sb = new java.lang.StringBuilder("MapFileInfo(");
+    java.lang.StringBuilder sb = new java.lang.StringBuilder("DataFileInfo(");
     boolean first = true;
 
     sb.append("estimatedSize:");
@@ -307,17 +307,17 @@
     }
   }
 
-  private static class MapFileInfoStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+  private static class DataFileInfoStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
     @Override
-    public MapFileInfoStandardScheme getScheme() {
-      return new MapFileInfoStandardScheme();
+    public DataFileInfoStandardScheme getScheme() {
+      return new DataFileInfoStandardScheme();
     }
   }
 
-  private static class MapFileInfoStandardScheme extends org.apache.thrift.scheme.StandardScheme<MapFileInfo> {
+  private static class DataFileInfoStandardScheme extends org.apache.thrift.scheme.StandardScheme<DataFileInfo> {
 
     @Override
-    public void read(org.apache.thrift.protocol.TProtocol iprot, MapFileInfo struct) throws org.apache.thrift.TException {
+    public void read(org.apache.thrift.protocol.TProtocol iprot, DataFileInfo struct) throws org.apache.thrift.TException {
       org.apache.thrift.protocol.TField schemeField;
       iprot.readStructBegin();
       while (true)
@@ -347,7 +347,7 @@
     }
 
     @Override
-    public void write(org.apache.thrift.protocol.TProtocol oprot, MapFileInfo struct) throws org.apache.thrift.TException {
+    public void write(org.apache.thrift.protocol.TProtocol oprot, DataFileInfo struct) throws org.apache.thrift.TException {
       struct.validate();
 
       oprot.writeStructBegin(STRUCT_DESC);
@@ -360,17 +360,17 @@
 
   }
 
-  private static class MapFileInfoTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+  private static class DataFileInfoTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
     @Override
-    public MapFileInfoTupleScheme getScheme() {
-      return new MapFileInfoTupleScheme();
+    public DataFileInfoTupleScheme getScheme() {
+      return new DataFileInfoTupleScheme();
     }
   }
 
-  private static class MapFileInfoTupleScheme extends org.apache.thrift.scheme.TupleScheme<MapFileInfo> {
+  private static class DataFileInfoTupleScheme extends org.apache.thrift.scheme.TupleScheme<DataFileInfo> {
 
     @Override
-    public void write(org.apache.thrift.protocol.TProtocol prot, MapFileInfo struct) throws org.apache.thrift.TException {
+    public void write(org.apache.thrift.protocol.TProtocol prot, DataFileInfo 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.isSetEstimatedSize()) {
@@ -383,7 +383,7 @@
     }
 
     @Override
-    public void read(org.apache.thrift.protocol.TProtocol prot, MapFileInfo struct) throws org.apache.thrift.TException {
+    public void read(org.apache.thrift.protocol.TProtocol prot, DataFileInfo 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)) {
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TDurability.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletingest/thrift/TDurability.java
similarity index 96%
rename from core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TDurability.java
rename to core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletingest/thrift/TDurability.java
index b5e8a95..2bab344 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TDurability.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletingest/thrift/TDurability.java
@@ -22,7 +22,7 @@
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
  */
-package org.apache.accumulo.core.tabletserver.thrift;
+package org.apache.accumulo.core.tabletingest.thrift;
 
 
 public enum TDurability implements org.apache.thrift.TEnum {
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletingest/thrift/TabletIngestClientService.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletingest/thrift/TabletIngestClientService.java
new file mode 100644
index 0000000..881fe6e
--- /dev/null
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletingest/thrift/TabletIngestClientService.java
@@ -0,0 +1,10359 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.17.0)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.accumulo.core.tabletingest.thrift;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+public class TabletIngestClientService {
+
+  public interface Iface {
+
+    public long startUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TDurability durability) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException;
+
+    public void applyUpdates(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long updateID, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent keyExtent, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TMutation> mutations) throws org.apache.thrift.TException;
+
+    public org.apache.accumulo.core.dataImpl.thrift.UpdateErrors closeUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long updateID) throws org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException, org.apache.thrift.TException;
+
+    public boolean cancelUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long updateID) throws org.apache.thrift.TException;
+
+    public org.apache.accumulo.core.dataImpl.thrift.TConditionalSession startConditionalUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.List<java.nio.ByteBuffer> authorizations, java.lang.String tableID, TDurability durability, java.lang.String classLoaderContext) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException;
+
+    public java.util.List<org.apache.accumulo.core.dataImpl.thrift.TCMResult> conditionalUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long sessID, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>> mutations, java.util.List<java.lang.String> symbols) throws org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException, org.apache.thrift.TException;
+
+    public void invalidateConditionalUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long sessID) throws org.apache.thrift.TException;
+
+    public void closeConditionalUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long sessID) throws org.apache.thrift.TException;
+
+    public void loadFiles(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long tid, java.lang.String dir, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,DataFileInfo>> files, boolean setTime) throws org.apache.thrift.TException;
+
+  }
+
+  public interface AsyncIface {
+
+    public void startUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TDurability durability, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler) throws org.apache.thrift.TException;
+
+    public void applyUpdates(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long updateID, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent keyExtent, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TMutation> mutations, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+
+    public void closeUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long updateID, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.UpdateErrors> resultHandler) throws org.apache.thrift.TException;
+
+    public void cancelUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long updateID, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException;
+
+    public void startConditionalUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.List<java.nio.ByteBuffer> authorizations, java.lang.String tableID, TDurability durability, java.lang.String classLoaderContext, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TConditionalSession> resultHandler) throws org.apache.thrift.TException;
+
+    public void conditionalUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long sessID, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>> mutations, java.util.List<java.lang.String> symbols, org.apache.thrift.async.AsyncMethodCallback<java.util.List<org.apache.accumulo.core.dataImpl.thrift.TCMResult>> resultHandler) throws org.apache.thrift.TException;
+
+    public void invalidateConditionalUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long sessID, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+
+    public void closeConditionalUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long sessID, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+
+    public void loadFiles(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long tid, java.lang.String dir, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,DataFileInfo>> files, boolean setTime, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+
+  }
+
+  public static class Client extends org.apache.thrift.TServiceClient implements Iface {
+    public static class Factory implements org.apache.thrift.TServiceClientFactory<Client> {
+      public Factory() {}
+      @Override
+      public Client getClient(org.apache.thrift.protocol.TProtocol prot) {
+        return new Client(prot);
+      }
+      @Override
+      public Client getClient(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TProtocol oprot) {
+        return new Client(iprot, oprot);
+      }
+    }
+
+    public Client(org.apache.thrift.protocol.TProtocol prot)
+    {
+      super(prot, prot);
+    }
+
+    public Client(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TProtocol oprot) {
+      super(iprot, oprot);
+    }
+
+    @Override
+    public long startUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TDurability durability) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
+    {
+      send_startUpdate(tinfo, credentials, durability);
+      return recv_startUpdate();
+    }
+
+    public void send_startUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TDurability durability) throws org.apache.thrift.TException
+    {
+      startUpdate_args args = new startUpdate_args();
+      args.setTinfo(tinfo);
+      args.setCredentials(credentials);
+      args.setDurability(durability);
+      sendBase("startUpdate", args);
+    }
+
+    public long recv_startUpdate() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
+    {
+      startUpdate_result result = new startUpdate_result();
+      receiveBase(result, "startUpdate");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.sec != null) {
+        throw result.sec;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "startUpdate failed: unknown result");
+    }
+
+    @Override
+    public void applyUpdates(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long updateID, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent keyExtent, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TMutation> mutations) throws org.apache.thrift.TException
+    {
+      send_applyUpdates(tinfo, updateID, keyExtent, mutations);
+    }
+
+    public void send_applyUpdates(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long updateID, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent keyExtent, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TMutation> mutations) throws org.apache.thrift.TException
+    {
+      applyUpdates_args args = new applyUpdates_args();
+      args.setTinfo(tinfo);
+      args.setUpdateID(updateID);
+      args.setKeyExtent(keyExtent);
+      args.setMutations(mutations);
+      sendBaseOneway("applyUpdates", args);
+    }
+
+    @Override
+    public org.apache.accumulo.core.dataImpl.thrift.UpdateErrors closeUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long updateID) throws org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException, org.apache.thrift.TException
+    {
+      send_closeUpdate(tinfo, updateID);
+      return recv_closeUpdate();
+    }
+
+    public void send_closeUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long updateID) throws org.apache.thrift.TException
+    {
+      closeUpdate_args args = new closeUpdate_args();
+      args.setTinfo(tinfo);
+      args.setUpdateID(updateID);
+      sendBase("closeUpdate", args);
+    }
+
+    public org.apache.accumulo.core.dataImpl.thrift.UpdateErrors recv_closeUpdate() throws org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException, org.apache.thrift.TException
+    {
+      closeUpdate_result result = new closeUpdate_result();
+      receiveBase(result, "closeUpdate");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.nssi != null) {
+        throw result.nssi;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "closeUpdate failed: unknown result");
+    }
+
+    @Override
+    public boolean cancelUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long updateID) throws org.apache.thrift.TException
+    {
+      send_cancelUpdate(tinfo, updateID);
+      return recv_cancelUpdate();
+    }
+
+    public void send_cancelUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long updateID) throws org.apache.thrift.TException
+    {
+      cancelUpdate_args args = new cancelUpdate_args();
+      args.setTinfo(tinfo);
+      args.setUpdateID(updateID);
+      sendBase("cancelUpdate", args);
+    }
+
+    public boolean recv_cancelUpdate() throws org.apache.thrift.TException
+    {
+      cancelUpdate_result result = new cancelUpdate_result();
+      receiveBase(result, "cancelUpdate");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "cancelUpdate failed: unknown result");
+    }
+
+    @Override
+    public org.apache.accumulo.core.dataImpl.thrift.TConditionalSession startConditionalUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.List<java.nio.ByteBuffer> authorizations, java.lang.String tableID, TDurability durability, java.lang.String classLoaderContext) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
+    {
+      send_startConditionalUpdate(tinfo, credentials, authorizations, tableID, durability, classLoaderContext);
+      return recv_startConditionalUpdate();
+    }
+
+    public void send_startConditionalUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.List<java.nio.ByteBuffer> authorizations, java.lang.String tableID, TDurability durability, java.lang.String classLoaderContext) throws org.apache.thrift.TException
+    {
+      startConditionalUpdate_args args = new startConditionalUpdate_args();
+      args.setTinfo(tinfo);
+      args.setCredentials(credentials);
+      args.setAuthorizations(authorizations);
+      args.setTableID(tableID);
+      args.setDurability(durability);
+      args.setClassLoaderContext(classLoaderContext);
+      sendBase("startConditionalUpdate", args);
+    }
+
+    public org.apache.accumulo.core.dataImpl.thrift.TConditionalSession recv_startConditionalUpdate() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
+    {
+      startConditionalUpdate_result result = new startConditionalUpdate_result();
+      receiveBase(result, "startConditionalUpdate");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.sec != null) {
+        throw result.sec;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "startConditionalUpdate failed: unknown result");
+    }
+
+    @Override
+    public java.util.List<org.apache.accumulo.core.dataImpl.thrift.TCMResult> conditionalUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long sessID, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>> mutations, java.util.List<java.lang.String> symbols) throws org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException, org.apache.thrift.TException
+    {
+      send_conditionalUpdate(tinfo, sessID, mutations, symbols);
+      return recv_conditionalUpdate();
+    }
+
+    public void send_conditionalUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long sessID, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>> mutations, java.util.List<java.lang.String> symbols) throws org.apache.thrift.TException
+    {
+      conditionalUpdate_args args = new conditionalUpdate_args();
+      args.setTinfo(tinfo);
+      args.setSessID(sessID);
+      args.setMutations(mutations);
+      args.setSymbols(symbols);
+      sendBase("conditionalUpdate", args);
+    }
+
+    public java.util.List<org.apache.accumulo.core.dataImpl.thrift.TCMResult> recv_conditionalUpdate() throws org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException, org.apache.thrift.TException
+    {
+      conditionalUpdate_result result = new conditionalUpdate_result();
+      receiveBase(result, "conditionalUpdate");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.nssi != null) {
+        throw result.nssi;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "conditionalUpdate failed: unknown result");
+    }
+
+    @Override
+    public void invalidateConditionalUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long sessID) throws org.apache.thrift.TException
+    {
+      send_invalidateConditionalUpdate(tinfo, sessID);
+      recv_invalidateConditionalUpdate();
+    }
+
+    public void send_invalidateConditionalUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long sessID) throws org.apache.thrift.TException
+    {
+      invalidateConditionalUpdate_args args = new invalidateConditionalUpdate_args();
+      args.setTinfo(tinfo);
+      args.setSessID(sessID);
+      sendBase("invalidateConditionalUpdate", args);
+    }
+
+    public void recv_invalidateConditionalUpdate() throws org.apache.thrift.TException
+    {
+      invalidateConditionalUpdate_result result = new invalidateConditionalUpdate_result();
+      receiveBase(result, "invalidateConditionalUpdate");
+      return;
+    }
+
+    @Override
+    public void closeConditionalUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long sessID) throws org.apache.thrift.TException
+    {
+      send_closeConditionalUpdate(tinfo, sessID);
+    }
+
+    public void send_closeConditionalUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long sessID) throws org.apache.thrift.TException
+    {
+      closeConditionalUpdate_args args = new closeConditionalUpdate_args();
+      args.setTinfo(tinfo);
+      args.setSessID(sessID);
+      sendBaseOneway("closeConditionalUpdate", args);
+    }
+
+    @Override
+    public void loadFiles(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long tid, java.lang.String dir, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,DataFileInfo>> files, boolean setTime) throws org.apache.thrift.TException
+    {
+      send_loadFiles(tinfo, credentials, tid, dir, files, setTime);
+    }
+
+    public void send_loadFiles(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long tid, java.lang.String dir, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,DataFileInfo>> files, boolean setTime) throws org.apache.thrift.TException
+    {
+      loadFiles_args args = new loadFiles_args();
+      args.setTinfo(tinfo);
+      args.setCredentials(credentials);
+      args.setTid(tid);
+      args.setDir(dir);
+      args.setFiles(files);
+      args.setSetTime(setTime);
+      sendBaseOneway("loadFiles", args);
+    }
+
+  }
+  public static class AsyncClient extends org.apache.thrift.async.TAsyncClient implements AsyncIface {
+    public static class Factory implements org.apache.thrift.async.TAsyncClientFactory<AsyncClient> {
+      private org.apache.thrift.async.TAsyncClientManager clientManager;
+      private org.apache.thrift.protocol.TProtocolFactory protocolFactory;
+      public Factory(org.apache.thrift.async.TAsyncClientManager clientManager, org.apache.thrift.protocol.TProtocolFactory protocolFactory) {
+        this.clientManager = clientManager;
+        this.protocolFactory = protocolFactory;
+      }
+    @Override
+      public AsyncClient getAsyncClient(org.apache.thrift.transport.TNonblockingTransport transport) {
+        return new AsyncClient(protocolFactory, clientManager, transport);
+      }
+    }
+
+    public AsyncClient(org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.async.TAsyncClientManager clientManager, org.apache.thrift.transport.TNonblockingTransport transport) {
+      super(protocolFactory, clientManager, transport);
+    }
+
+    @Override
+    public void startUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TDurability durability, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      startUpdate_call method_call = new startUpdate_call(tinfo, credentials, durability, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class startUpdate_call extends org.apache.thrift.async.TAsyncMethodCall<java.lang.Long> {
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
+      private TDurability durability;
+      public startUpdate_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TDurability durability, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> 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.durability = durability;
+      }
+
+      @Override
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("startUpdate", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        startUpdate_args args = new startUpdate_args();
+        args.setTinfo(tinfo);
+        args.setCredentials(credentials);
+        args.setDurability(durability);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      @Override
+      public java.lang.Long getResult() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, 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_startUpdate();
+      }
+    }
+
+    @Override
+    public void applyUpdates(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long updateID, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent keyExtent, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TMutation> mutations, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      applyUpdates_call method_call = new applyUpdates_call(tinfo, updateID, keyExtent, mutations, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class applyUpdates_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
+      private long updateID;
+      private org.apache.accumulo.core.dataImpl.thrift.TKeyExtent keyExtent;
+      private java.util.List<org.apache.accumulo.core.dataImpl.thrift.TMutation> mutations;
+      public applyUpdates_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long updateID, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent keyExtent, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TMutation> mutations, org.apache.thrift.async.AsyncMethodCallback<Void> 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, true);
+        this.tinfo = tinfo;
+        this.updateID = updateID;
+        this.keyExtent = keyExtent;
+        this.mutations = mutations;
+      }
+
+      @Override
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("applyUpdates", org.apache.thrift.protocol.TMessageType.ONEWAY, 0));
+        applyUpdates_args args = new applyUpdates_args();
+        args.setTinfo(tinfo);
+        args.setUpdateID(updateID);
+        args.setKeyExtent(keyExtent);
+        args.setMutations(mutations);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      @Override
+      public Void 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 null;
+      }
+    }
+
+    @Override
+    public void closeUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long updateID, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.UpdateErrors> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      closeUpdate_call method_call = new closeUpdate_call(tinfo, updateID, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class closeUpdate_call extends org.apache.thrift.async.TAsyncMethodCall<org.apache.accumulo.core.dataImpl.thrift.UpdateErrors> {
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
+      private long updateID;
+      public closeUpdate_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long updateID, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.UpdateErrors> 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.updateID = updateID;
+      }
+
+      @Override
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("closeUpdate", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        closeUpdate_args args = new closeUpdate_args();
+        args.setTinfo(tinfo);
+        args.setUpdateID(updateID);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      @Override
+      public org.apache.accumulo.core.dataImpl.thrift.UpdateErrors getResult() throws org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException, 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_closeUpdate();
+      }
+    }
+
+    @Override
+    public void cancelUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long updateID, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      cancelUpdate_call method_call = new cancelUpdate_call(tinfo, updateID, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class cancelUpdate_call extends org.apache.thrift.async.TAsyncMethodCall<java.lang.Boolean> {
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
+      private long updateID;
+      public cancelUpdate_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long updateID, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> 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.updateID = updateID;
+      }
+
+      @Override
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("cancelUpdate", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        cancelUpdate_args args = new cancelUpdate_args();
+        args.setTinfo(tinfo);
+        args.setUpdateID(updateID);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      @Override
+      public java.lang.Boolean 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_cancelUpdate();
+      }
+    }
+
+    @Override
+    public void startConditionalUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.List<java.nio.ByteBuffer> authorizations, java.lang.String tableID, TDurability durability, java.lang.String classLoaderContext, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TConditionalSession> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      startConditionalUpdate_call method_call = new startConditionalUpdate_call(tinfo, credentials, authorizations, tableID, durability, classLoaderContext, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class startConditionalUpdate_call extends org.apache.thrift.async.TAsyncMethodCall<org.apache.accumulo.core.dataImpl.thrift.TConditionalSession> {
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
+      private java.util.List<java.nio.ByteBuffer> authorizations;
+      private java.lang.String tableID;
+      private TDurability durability;
+      private java.lang.String classLoaderContext;
+      public startConditionalUpdate_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.List<java.nio.ByteBuffer> authorizations, java.lang.String tableID, TDurability durability, java.lang.String classLoaderContext, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TConditionalSession> 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.authorizations = authorizations;
+        this.tableID = tableID;
+        this.durability = durability;
+        this.classLoaderContext = classLoaderContext;
+      }
+
+      @Override
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("startConditionalUpdate", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        startConditionalUpdate_args args = new startConditionalUpdate_args();
+        args.setTinfo(tinfo);
+        args.setCredentials(credentials);
+        args.setAuthorizations(authorizations);
+        args.setTableID(tableID);
+        args.setDurability(durability);
+        args.setClassLoaderContext(classLoaderContext);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      @Override
+      public org.apache.accumulo.core.dataImpl.thrift.TConditionalSession getResult() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, 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_startConditionalUpdate();
+      }
+    }
+
+    @Override
+    public void conditionalUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long sessID, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>> mutations, java.util.List<java.lang.String> symbols, org.apache.thrift.async.AsyncMethodCallback<java.util.List<org.apache.accumulo.core.dataImpl.thrift.TCMResult>> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      conditionalUpdate_call method_call = new conditionalUpdate_call(tinfo, sessID, mutations, symbols, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class conditionalUpdate_call extends org.apache.thrift.async.TAsyncMethodCall<java.util.List<org.apache.accumulo.core.dataImpl.thrift.TCMResult>> {
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
+      private long sessID;
+      private java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>> mutations;
+      private java.util.List<java.lang.String> symbols;
+      public conditionalUpdate_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long sessID, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>> mutations, java.util.List<java.lang.String> symbols, org.apache.thrift.async.AsyncMethodCallback<java.util.List<org.apache.accumulo.core.dataImpl.thrift.TCMResult>> 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.sessID = sessID;
+        this.mutations = mutations;
+        this.symbols = symbols;
+      }
+
+      @Override
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("conditionalUpdate", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        conditionalUpdate_args args = new conditionalUpdate_args();
+        args.setTinfo(tinfo);
+        args.setSessID(sessID);
+        args.setMutations(mutations);
+        args.setSymbols(symbols);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      @Override
+      public java.util.List<org.apache.accumulo.core.dataImpl.thrift.TCMResult> getResult() throws org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException, 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_conditionalUpdate();
+      }
+    }
+
+    @Override
+    public void invalidateConditionalUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long sessID, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      invalidateConditionalUpdate_call method_call = new invalidateConditionalUpdate_call(tinfo, sessID, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class invalidateConditionalUpdate_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
+      private long sessID;
+      public invalidateConditionalUpdate_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long sessID, org.apache.thrift.async.AsyncMethodCallback<Void> 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.sessID = sessID;
+      }
+
+      @Override
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("invalidateConditionalUpdate", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        invalidateConditionalUpdate_args args = new invalidateConditionalUpdate_args();
+        args.setTinfo(tinfo);
+        args.setSessID(sessID);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      @Override
+      public Void 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);
+        (new Client(prot)).recv_invalidateConditionalUpdate();
+        return null;
+      }
+    }
+
+    @Override
+    public void closeConditionalUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long sessID, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      closeConditionalUpdate_call method_call = new closeConditionalUpdate_call(tinfo, sessID, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class closeConditionalUpdate_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
+      private long sessID;
+      public closeConditionalUpdate_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long sessID, org.apache.thrift.async.AsyncMethodCallback<Void> 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, true);
+        this.tinfo = tinfo;
+        this.sessID = sessID;
+      }
+
+      @Override
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("closeConditionalUpdate", org.apache.thrift.protocol.TMessageType.ONEWAY, 0));
+        closeConditionalUpdate_args args = new closeConditionalUpdate_args();
+        args.setTinfo(tinfo);
+        args.setSessID(sessID);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      @Override
+      public Void 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 null;
+      }
+    }
+
+    @Override
+    public void loadFiles(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long tid, java.lang.String dir, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,DataFileInfo>> files, boolean setTime, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      loadFiles_call method_call = new loadFiles_call(tinfo, credentials, tid, dir, files, setTime, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class loadFiles_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
+      private long tid;
+      private java.lang.String dir;
+      private java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,DataFileInfo>> files;
+      private boolean setTime;
+      public loadFiles_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long tid, java.lang.String dir, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,DataFileInfo>> files, boolean setTime, org.apache.thrift.async.AsyncMethodCallback<Void> 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, true);
+        this.tinfo = tinfo;
+        this.credentials = credentials;
+        this.tid = tid;
+        this.dir = dir;
+        this.files = files;
+        this.setTime = setTime;
+      }
+
+      @Override
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("loadFiles", org.apache.thrift.protocol.TMessageType.ONEWAY, 0));
+        loadFiles_args args = new loadFiles_args();
+        args.setTinfo(tinfo);
+        args.setCredentials(credentials);
+        args.setTid(tid);
+        args.setDir(dir);
+        args.setFiles(files);
+        args.setSetTime(setTime);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      @Override
+      public Void 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 null;
+      }
+    }
+
+  }
+
+  public static class Processor<I extends Iface> extends org.apache.thrift.TBaseProcessor<I> implements org.apache.thrift.TProcessor {
+    private static final org.slf4j.Logger _LOGGER = org.slf4j.LoggerFactory.getLogger(Processor.class.getName());
+    public Processor(I iface) {
+      super(iface, getProcessMap(new java.util.HashMap<java.lang.String, org.apache.thrift.ProcessFunction<I, ? extends org.apache.thrift.TBase>>()));
+    }
+
+    protected Processor(I iface, java.util.Map<java.lang.String, org.apache.thrift.ProcessFunction<I, ? extends org.apache.thrift.TBase>> processMap) {
+      super(iface, getProcessMap(processMap));
+    }
+
+    private static <I extends Iface> java.util.Map<java.lang.String,  org.apache.thrift.ProcessFunction<I, ? extends org.apache.thrift.TBase>> getProcessMap(java.util.Map<java.lang.String, org.apache.thrift.ProcessFunction<I, ? extends  org.apache.thrift.TBase>> processMap) {
+      processMap.put("startUpdate", new startUpdate());
+      processMap.put("applyUpdates", new applyUpdates());
+      processMap.put("closeUpdate", new closeUpdate());
+      processMap.put("cancelUpdate", new cancelUpdate());
+      processMap.put("startConditionalUpdate", new startConditionalUpdate());
+      processMap.put("conditionalUpdate", new conditionalUpdate());
+      processMap.put("invalidateConditionalUpdate", new invalidateConditionalUpdate());
+      processMap.put("closeConditionalUpdate", new closeConditionalUpdate());
+      processMap.put("loadFiles", new loadFiles());
+      return processMap;
+    }
+
+    public static class startUpdate<I extends Iface> extends org.apache.thrift.ProcessFunction<I, startUpdate_args> {
+      public startUpdate() {
+        super("startUpdate");
+      }
+
+      @Override
+      public startUpdate_args getEmptyArgsInstance() {
+        return new startUpdate_args();
+      }
+
+      @Override
+      protected boolean isOneway() {
+        return false;
+      }
+
+      @Override
+      protected boolean rethrowUnhandledExceptions() {
+        return false;
+      }
+
+      @Override
+      public startUpdate_result getResult(I iface, startUpdate_args args) throws org.apache.thrift.TException {
+        startUpdate_result result = new startUpdate_result();
+        try {
+          result.success = iface.startUpdate(args.tinfo, args.credentials, args.durability);
+          result.setSuccessIsSet(true);
+        } catch (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) {
+          result.sec = sec;
+        }
+        return result;
+      }
+    }
+
+    public static class applyUpdates<I extends Iface> extends org.apache.thrift.ProcessFunction<I, applyUpdates_args> {
+      public applyUpdates() {
+        super("applyUpdates");
+      }
+
+      @Override
+      public applyUpdates_args getEmptyArgsInstance() {
+        return new applyUpdates_args();
+      }
+
+      @Override
+      protected boolean isOneway() {
+        return true;
+      }
+
+      @Override
+      protected boolean rethrowUnhandledExceptions() {
+        return false;
+      }
+
+      @Override
+      public org.apache.thrift.TBase getResult(I iface, applyUpdates_args args) throws org.apache.thrift.TException {
+        iface.applyUpdates(args.tinfo, args.updateID, args.keyExtent, args.mutations);
+        return null;
+      }
+    }
+
+    public static class closeUpdate<I extends Iface> extends org.apache.thrift.ProcessFunction<I, closeUpdate_args> {
+      public closeUpdate() {
+        super("closeUpdate");
+      }
+
+      @Override
+      public closeUpdate_args getEmptyArgsInstance() {
+        return new closeUpdate_args();
+      }
+
+      @Override
+      protected boolean isOneway() {
+        return false;
+      }
+
+      @Override
+      protected boolean rethrowUnhandledExceptions() {
+        return false;
+      }
+
+      @Override
+      public closeUpdate_result getResult(I iface, closeUpdate_args args) throws org.apache.thrift.TException {
+        closeUpdate_result result = new closeUpdate_result();
+        try {
+          result.success = iface.closeUpdate(args.tinfo, args.updateID);
+        } catch (org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException nssi) {
+          result.nssi = nssi;
+        }
+        return result;
+      }
+    }
+
+    public static class cancelUpdate<I extends Iface> extends org.apache.thrift.ProcessFunction<I, cancelUpdate_args> {
+      public cancelUpdate() {
+        super("cancelUpdate");
+      }
+
+      @Override
+      public cancelUpdate_args getEmptyArgsInstance() {
+        return new cancelUpdate_args();
+      }
+
+      @Override
+      protected boolean isOneway() {
+        return false;
+      }
+
+      @Override
+      protected boolean rethrowUnhandledExceptions() {
+        return false;
+      }
+
+      @Override
+      public cancelUpdate_result getResult(I iface, cancelUpdate_args args) throws org.apache.thrift.TException {
+        cancelUpdate_result result = new cancelUpdate_result();
+        result.success = iface.cancelUpdate(args.tinfo, args.updateID);
+        result.setSuccessIsSet(true);
+        return result;
+      }
+    }
+
+    public static class startConditionalUpdate<I extends Iface> extends org.apache.thrift.ProcessFunction<I, startConditionalUpdate_args> {
+      public startConditionalUpdate() {
+        super("startConditionalUpdate");
+      }
+
+      @Override
+      public startConditionalUpdate_args getEmptyArgsInstance() {
+        return new startConditionalUpdate_args();
+      }
+
+      @Override
+      protected boolean isOneway() {
+        return false;
+      }
+
+      @Override
+      protected boolean rethrowUnhandledExceptions() {
+        return false;
+      }
+
+      @Override
+      public startConditionalUpdate_result getResult(I iface, startConditionalUpdate_args args) throws org.apache.thrift.TException {
+        startConditionalUpdate_result result = new startConditionalUpdate_result();
+        try {
+          result.success = iface.startConditionalUpdate(args.tinfo, args.credentials, args.authorizations, args.tableID, args.durability, args.classLoaderContext);
+        } catch (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) {
+          result.sec = sec;
+        }
+        return result;
+      }
+    }
+
+    public static class conditionalUpdate<I extends Iface> extends org.apache.thrift.ProcessFunction<I, conditionalUpdate_args> {
+      public conditionalUpdate() {
+        super("conditionalUpdate");
+      }
+
+      @Override
+      public conditionalUpdate_args getEmptyArgsInstance() {
+        return new conditionalUpdate_args();
+      }
+
+      @Override
+      protected boolean isOneway() {
+        return false;
+      }
+
+      @Override
+      protected boolean rethrowUnhandledExceptions() {
+        return false;
+      }
+
+      @Override
+      public conditionalUpdate_result getResult(I iface, conditionalUpdate_args args) throws org.apache.thrift.TException {
+        conditionalUpdate_result result = new conditionalUpdate_result();
+        try {
+          result.success = iface.conditionalUpdate(args.tinfo, args.sessID, args.mutations, args.symbols);
+        } catch (org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException nssi) {
+          result.nssi = nssi;
+        }
+        return result;
+      }
+    }
+
+    public static class invalidateConditionalUpdate<I extends Iface> extends org.apache.thrift.ProcessFunction<I, invalidateConditionalUpdate_args> {
+      public invalidateConditionalUpdate() {
+        super("invalidateConditionalUpdate");
+      }
+
+      @Override
+      public invalidateConditionalUpdate_args getEmptyArgsInstance() {
+        return new invalidateConditionalUpdate_args();
+      }
+
+      @Override
+      protected boolean isOneway() {
+        return false;
+      }
+
+      @Override
+      protected boolean rethrowUnhandledExceptions() {
+        return false;
+      }
+
+      @Override
+      public invalidateConditionalUpdate_result getResult(I iface, invalidateConditionalUpdate_args args) throws org.apache.thrift.TException {
+        invalidateConditionalUpdate_result result = new invalidateConditionalUpdate_result();
+        iface.invalidateConditionalUpdate(args.tinfo, args.sessID);
+        return result;
+      }
+    }
+
+    public static class closeConditionalUpdate<I extends Iface> extends org.apache.thrift.ProcessFunction<I, closeConditionalUpdate_args> {
+      public closeConditionalUpdate() {
+        super("closeConditionalUpdate");
+      }
+
+      @Override
+      public closeConditionalUpdate_args getEmptyArgsInstance() {
+        return new closeConditionalUpdate_args();
+      }
+
+      @Override
+      protected boolean isOneway() {
+        return true;
+      }
+
+      @Override
+      protected boolean rethrowUnhandledExceptions() {
+        return false;
+      }
+
+      @Override
+      public org.apache.thrift.TBase getResult(I iface, closeConditionalUpdate_args args) throws org.apache.thrift.TException {
+        iface.closeConditionalUpdate(args.tinfo, args.sessID);
+        return null;
+      }
+    }
+
+    public static class loadFiles<I extends Iface> extends org.apache.thrift.ProcessFunction<I, loadFiles_args> {
+      public loadFiles() {
+        super("loadFiles");
+      }
+
+      @Override
+      public loadFiles_args getEmptyArgsInstance() {
+        return new loadFiles_args();
+      }
+
+      @Override
+      protected boolean isOneway() {
+        return true;
+      }
+
+      @Override
+      protected boolean rethrowUnhandledExceptions() {
+        return false;
+      }
+
+      @Override
+      public org.apache.thrift.TBase getResult(I iface, loadFiles_args args) throws org.apache.thrift.TException {
+        iface.loadFiles(args.tinfo, args.credentials, args.tid, args.dir, args.files, args.setTime);
+        return null;
+      }
+    }
+
+  }
+
+  public static class AsyncProcessor<I extends AsyncIface> extends org.apache.thrift.TBaseAsyncProcessor<I> {
+    private static final org.slf4j.Logger _LOGGER = org.slf4j.LoggerFactory.getLogger(AsyncProcessor.class.getName());
+    public AsyncProcessor(I iface) {
+      super(iface, getProcessMap(new java.util.HashMap<java.lang.String, org.apache.thrift.AsyncProcessFunction<I, ? extends org.apache.thrift.TBase, ?>>()));
+    }
+
+    protected AsyncProcessor(I iface, java.util.Map<java.lang.String,  org.apache.thrift.AsyncProcessFunction<I, ? extends  org.apache.thrift.TBase, ?>> processMap) {
+      super(iface, getProcessMap(processMap));
+    }
+
+    private static <I extends AsyncIface> java.util.Map<java.lang.String,  org.apache.thrift.AsyncProcessFunction<I, ? extends  org.apache.thrift.TBase,?>> getProcessMap(java.util.Map<java.lang.String,  org.apache.thrift.AsyncProcessFunction<I, ? extends  org.apache.thrift.TBase, ?>> processMap) {
+      processMap.put("startUpdate", new startUpdate());
+      processMap.put("applyUpdates", new applyUpdates());
+      processMap.put("closeUpdate", new closeUpdate());
+      processMap.put("cancelUpdate", new cancelUpdate());
+      processMap.put("startConditionalUpdate", new startConditionalUpdate());
+      processMap.put("conditionalUpdate", new conditionalUpdate());
+      processMap.put("invalidateConditionalUpdate", new invalidateConditionalUpdate());
+      processMap.put("closeConditionalUpdate", new closeConditionalUpdate());
+      processMap.put("loadFiles", new loadFiles());
+      return processMap;
+    }
+
+    public static class startUpdate<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, startUpdate_args, java.lang.Long> {
+      public startUpdate() {
+        super("startUpdate");
+      }
+
+      @Override
+      public startUpdate_args getEmptyArgsInstance() {
+        return new startUpdate_args();
+      }
+
+      @Override
+      public org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> 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.Long>() { 
+          @Override
+          public void onComplete(java.lang.Long o) {
+            startUpdate_result result = new startUpdate_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);
+            }
+          }
+          @Override
+          public void onError(java.lang.Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TSerializable msg;
+            startUpdate_result result = new startUpdate_result();
+            if (e instanceof org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) {
+              result.sec = (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) e;
+              result.setSecIsSet(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();
+            }
+          }
+        };
+      }
+
+      @Override
+      protected boolean isOneway() {
+        return false;
+      }
+
+      @Override
+      public void start(I iface, startUpdate_args args, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler) throws org.apache.thrift.TException {
+        iface.startUpdate(args.tinfo, args.credentials, args.durability,resultHandler);
+      }
+    }
+
+    public static class applyUpdates<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, applyUpdates_args, Void> {
+      public applyUpdates() {
+        super("applyUpdates");
+      }
+
+      @Override
+      public applyUpdates_args getEmptyArgsInstance() {
+        return new applyUpdates_args();
+      }
+
+      @Override
+      public org.apache.thrift.async.AsyncMethodCallback<Void> 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<Void>() { 
+          @Override
+          public void onComplete(Void o) {
+          }
+          @Override
+          public void onError(java.lang.Exception e) {
+            if (e instanceof org.apache.thrift.transport.TTransportException) {
+              _LOGGER.error("TTransportException inside handler", e);
+              fb.close();
+            } else {
+              _LOGGER.error("Exception inside oneway handler", e);
+            }
+          }
+        };
+      }
+
+      @Override
+      protected boolean isOneway() {
+        return true;
+      }
+
+      @Override
+      public void start(I iface, applyUpdates_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.applyUpdates(args.tinfo, args.updateID, args.keyExtent, args.mutations,resultHandler);
+      }
+    }
+
+    public static class closeUpdate<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, closeUpdate_args, org.apache.accumulo.core.dataImpl.thrift.UpdateErrors> {
+      public closeUpdate() {
+        super("closeUpdate");
+      }
+
+      @Override
+      public closeUpdate_args getEmptyArgsInstance() {
+        return new closeUpdate_args();
+      }
+
+      @Override
+      public org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.UpdateErrors> 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<org.apache.accumulo.core.dataImpl.thrift.UpdateErrors>() { 
+          @Override
+          public void onComplete(org.apache.accumulo.core.dataImpl.thrift.UpdateErrors o) {
+            closeUpdate_result result = new closeUpdate_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);
+            }
+          }
+          @Override
+          public void onError(java.lang.Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TSerializable msg;
+            closeUpdate_result result = new closeUpdate_result();
+            if (e instanceof org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException) {
+              result.nssi = (org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException) e;
+              result.setNssiIsSet(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();
+            }
+          }
+        };
+      }
+
+      @Override
+      protected boolean isOneway() {
+        return false;
+      }
+
+      @Override
+      public void start(I iface, closeUpdate_args args, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.UpdateErrors> resultHandler) throws org.apache.thrift.TException {
+        iface.closeUpdate(args.tinfo, args.updateID,resultHandler);
+      }
+    }
+
+    public static class cancelUpdate<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, cancelUpdate_args, java.lang.Boolean> {
+      public cancelUpdate() {
+        super("cancelUpdate");
+      }
+
+      @Override
+      public cancelUpdate_args getEmptyArgsInstance() {
+        return new cancelUpdate_args();
+      }
+
+      @Override
+      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>() { 
+          @Override
+          public void onComplete(java.lang.Boolean o) {
+            cancelUpdate_result result = new cancelUpdate_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);
+            }
+          }
+          @Override
+          public void onError(java.lang.Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TSerializable msg;
+            cancelUpdate_result result = new cancelUpdate_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();
+            }
+          }
+        };
+      }
+
+      @Override
+      protected boolean isOneway() {
+        return false;
+      }
+
+      @Override
+      public void start(I iface, cancelUpdate_args args, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
+        iface.cancelUpdate(args.tinfo, args.updateID,resultHandler);
+      }
+    }
+
+    public static class startConditionalUpdate<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, startConditionalUpdate_args, org.apache.accumulo.core.dataImpl.thrift.TConditionalSession> {
+      public startConditionalUpdate() {
+        super("startConditionalUpdate");
+      }
+
+      @Override
+      public startConditionalUpdate_args getEmptyArgsInstance() {
+        return new startConditionalUpdate_args();
+      }
+
+      @Override
+      public org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TConditionalSession> 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<org.apache.accumulo.core.dataImpl.thrift.TConditionalSession>() { 
+          @Override
+          public void onComplete(org.apache.accumulo.core.dataImpl.thrift.TConditionalSession o) {
+            startConditionalUpdate_result result = new startConditionalUpdate_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);
+            }
+          }
+          @Override
+          public void onError(java.lang.Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TSerializable msg;
+            startConditionalUpdate_result result = new startConditionalUpdate_result();
+            if (e instanceof org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) {
+              result.sec = (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) e;
+              result.setSecIsSet(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();
+            }
+          }
+        };
+      }
+
+      @Override
+      protected boolean isOneway() {
+        return false;
+      }
+
+      @Override
+      public void start(I iface, startConditionalUpdate_args args, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TConditionalSession> resultHandler) throws org.apache.thrift.TException {
+        iface.startConditionalUpdate(args.tinfo, args.credentials, args.authorizations, args.tableID, args.durability, args.classLoaderContext,resultHandler);
+      }
+    }
+
+    public static class conditionalUpdate<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, conditionalUpdate_args, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TCMResult>> {
+      public conditionalUpdate() {
+        super("conditionalUpdate");
+      }
+
+      @Override
+      public conditionalUpdate_args getEmptyArgsInstance() {
+        return new conditionalUpdate_args();
+      }
+
+      @Override
+      public org.apache.thrift.async.AsyncMethodCallback<java.util.List<org.apache.accumulo.core.dataImpl.thrift.TCMResult>> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new org.apache.thrift.async.AsyncMethodCallback<java.util.List<org.apache.accumulo.core.dataImpl.thrift.TCMResult>>() { 
+          @Override
+          public void onComplete(java.util.List<org.apache.accumulo.core.dataImpl.thrift.TCMResult> o) {
+            conditionalUpdate_result result = new conditionalUpdate_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);
+            }
+          }
+          @Override
+          public void onError(java.lang.Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TSerializable msg;
+            conditionalUpdate_result result = new conditionalUpdate_result();
+            if (e instanceof org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException) {
+              result.nssi = (org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException) e;
+              result.setNssiIsSet(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();
+            }
+          }
+        };
+      }
+
+      @Override
+      protected boolean isOneway() {
+        return false;
+      }
+
+      @Override
+      public void start(I iface, conditionalUpdate_args args, org.apache.thrift.async.AsyncMethodCallback<java.util.List<org.apache.accumulo.core.dataImpl.thrift.TCMResult>> resultHandler) throws org.apache.thrift.TException {
+        iface.conditionalUpdate(args.tinfo, args.sessID, args.mutations, args.symbols,resultHandler);
+      }
+    }
+
+    public static class invalidateConditionalUpdate<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, invalidateConditionalUpdate_args, Void> {
+      public invalidateConditionalUpdate() {
+        super("invalidateConditionalUpdate");
+      }
+
+      @Override
+      public invalidateConditionalUpdate_args getEmptyArgsInstance() {
+        return new invalidateConditionalUpdate_args();
+      }
+
+      @Override
+      public org.apache.thrift.async.AsyncMethodCallback<Void> 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<Void>() { 
+          @Override
+          public void onComplete(Void o) {
+            invalidateConditionalUpdate_result result = new invalidateConditionalUpdate_result();
+            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);
+            }
+          }
+          @Override
+          public void onError(java.lang.Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TSerializable msg;
+            invalidateConditionalUpdate_result result = new invalidateConditionalUpdate_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();
+            }
+          }
+        };
+      }
+
+      @Override
+      protected boolean isOneway() {
+        return false;
+      }
+
+      @Override
+      public void start(I iface, invalidateConditionalUpdate_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.invalidateConditionalUpdate(args.tinfo, args.sessID,resultHandler);
+      }
+    }
+
+    public static class closeConditionalUpdate<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, closeConditionalUpdate_args, Void> {
+      public closeConditionalUpdate() {
+        super("closeConditionalUpdate");
+      }
+
+      @Override
+      public closeConditionalUpdate_args getEmptyArgsInstance() {
+        return new closeConditionalUpdate_args();
+      }
+
+      @Override
+      public org.apache.thrift.async.AsyncMethodCallback<Void> 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<Void>() { 
+          @Override
+          public void onComplete(Void o) {
+          }
+          @Override
+          public void onError(java.lang.Exception e) {
+            if (e instanceof org.apache.thrift.transport.TTransportException) {
+              _LOGGER.error("TTransportException inside handler", e);
+              fb.close();
+            } else {
+              _LOGGER.error("Exception inside oneway handler", e);
+            }
+          }
+        };
+      }
+
+      @Override
+      protected boolean isOneway() {
+        return true;
+      }
+
+      @Override
+      public void start(I iface, closeConditionalUpdate_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.closeConditionalUpdate(args.tinfo, args.sessID,resultHandler);
+      }
+    }
+
+    public static class loadFiles<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, loadFiles_args, Void> {
+      public loadFiles() {
+        super("loadFiles");
+      }
+
+      @Override
+      public loadFiles_args getEmptyArgsInstance() {
+        return new loadFiles_args();
+      }
+
+      @Override
+      public org.apache.thrift.async.AsyncMethodCallback<Void> 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<Void>() { 
+          @Override
+          public void onComplete(Void o) {
+          }
+          @Override
+          public void onError(java.lang.Exception e) {
+            if (e instanceof org.apache.thrift.transport.TTransportException) {
+              _LOGGER.error("TTransportException inside handler", e);
+              fb.close();
+            } else {
+              _LOGGER.error("Exception inside oneway handler", e);
+            }
+          }
+        };
+      }
+
+      @Override
+      protected boolean isOneway() {
+        return true;
+      }
+
+      @Override
+      public void start(I iface, loadFiles_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.loadFiles(args.tinfo, args.credentials, args.tid, args.dir, args.files, args.setTime,resultHandler);
+      }
+    }
+
+  }
+
+  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+  public static class startUpdate_args implements org.apache.thrift.TBase<startUpdate_args, startUpdate_args._Fields>, java.io.Serializable, Cloneable, Comparable<startUpdate_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("startUpdate_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)1);
+    private static final org.apache.thrift.protocol.TField DURABILITY_FIELD_DESC = new org.apache.thrift.protocol.TField("durability", org.apache.thrift.protocol.TType.I32, (short)3);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new startUpdate_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new startUpdate_argsTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
+    /**
+     * 
+     * @see TDurability
+     */
+    public @org.apache.thrift.annotation.Nullable TDurability durability; // 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)1, "credentials"),
+      /**
+       * 
+       * @see TDurability
+       */
+      DURABILITY((short)3, "durability");
+
+      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 1: // CREDENTIALS
+            return CREDENTIALS;
+          case 3: // DURABILITY
+            return DURABILITY;
+          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;
+      }
+
+      @Override
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      @Override
+      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.clientImpl.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.DURABILITY, new org.apache.thrift.meta_data.FieldMetaData("durability", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TDurability.class)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(startUpdate_args.class, metaDataMap);
+    }
+
+    public startUpdate_args() {
+    }
+
+    public startUpdate_args(
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
+      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
+      TDurability durability)
+    {
+      this();
+      this.tinfo = tinfo;
+      this.credentials = credentials;
+      this.durability = durability;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public startUpdate_args(startUpdate_args other) {
+      if (other.isSetTinfo()) {
+        this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
+      }
+      if (other.isSetCredentials()) {
+        this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
+      }
+      if (other.isSetDurability()) {
+        this.durability = other.durability;
+      }
+    }
+
+    @Override
+    public startUpdate_args deepCopy() {
+      return new startUpdate_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.tinfo = null;
+      this.credentials = null;
+      this.durability = null;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
+      return this.tinfo;
+    }
+
+    public startUpdate_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.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 startUpdate_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 TDurability
+     */
+    @org.apache.thrift.annotation.Nullable
+    public TDurability getDurability() {
+      return this.durability;
+    }
+
+    /**
+     * 
+     * @see TDurability
+     */
+    public startUpdate_args setDurability(@org.apache.thrift.annotation.Nullable TDurability durability) {
+      this.durability = durability;
+      return this;
+    }
+
+    public void unsetDurability() {
+      this.durability = null;
+    }
+
+    /** Returns true if field durability is set (has been assigned a value) and false otherwise */
+    public boolean isSetDurability() {
+      return this.durability != null;
+    }
+
+    public void setDurabilityIsSet(boolean value) {
+      if (!value) {
+        this.durability = null;
+      }
+    }
+
+    @Override
+    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.clientImpl.thrift.TInfo)value);
+        }
+        break;
+
+      case CREDENTIALS:
+        if (value == null) {
+          unsetCredentials();
+        } else {
+          setCredentials((org.apache.accumulo.core.securityImpl.thrift.TCredentials)value);
+        }
+        break;
+
+      case DURABILITY:
+        if (value == null) {
+          unsetDurability();
+        } else {
+          setDurability((TDurability)value);
+        }
+        break;
+
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    @Override
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case TINFO:
+        return getTinfo();
+
+      case CREDENTIALS:
+        return getCredentials();
+
+      case DURABILITY:
+        return getDurability();
+
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    @Override
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case TINFO:
+        return isSetTinfo();
+      case CREDENTIALS:
+        return isSetCredentials();
+      case DURABILITY:
+        return isSetDurability();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that instanceof startUpdate_args)
+        return this.equals((startUpdate_args)that);
+      return false;
+    }
+
+    public boolean equals(startUpdate_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_durability = true && this.isSetDurability();
+      boolean that_present_durability = true && that.isSetDurability();
+      if (this_present_durability || that_present_durability) {
+        if (!(this_present_durability && that_present_durability))
+          return false;
+        if (!this.durability.equals(that.durability))
+          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 + ((isSetDurability()) ? 131071 : 524287);
+      if (isSetDurability())
+        hashCode = hashCode * 8191 + durability.getValue();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(startUpdate_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(isSetDurability(), other.isSetDurability());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetDurability()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.durability, other.durability);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    @Override
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      scheme(iprot).read(iprot, this);
+    }
+
+    @Override
+    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("startUpdate_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("durability:");
+      if (this.durability == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.durability);
+      }
+      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 startUpdate_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public startUpdate_argsStandardScheme getScheme() {
+        return new startUpdate_argsStandardScheme();
+      }
+    }
+
+    private static class startUpdate_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<startUpdate_args> {
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol iprot, startUpdate_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.clientImpl.thrift.TInfo();
+                struct.tinfo.read(iprot);
+                struct.setTinfoIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 1: // 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 3: // DURABILITY
+              if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+                struct.durability = org.apache.accumulo.core.tabletingest.thrift.TDurability.findByValue(iprot.readI32());
+                struct.setDurabilityIsSet(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();
+      }
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol oprot, startUpdate_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.credentials != null) {
+          oprot.writeFieldBegin(CREDENTIALS_FIELD_DESC);
+          struct.credentials.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.tinfo != null) {
+          oprot.writeFieldBegin(TINFO_FIELD_DESC);
+          struct.tinfo.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.durability != null) {
+          oprot.writeFieldBegin(DURABILITY_FIELD_DESC);
+          oprot.writeI32(struct.durability.getValue());
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class startUpdate_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public startUpdate_argsTupleScheme getScheme() {
+        return new startUpdate_argsTupleScheme();
+      }
+    }
+
+    private static class startUpdate_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<startUpdate_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, startUpdate_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.isSetDurability()) {
+          optionals.set(2);
+        }
+        oprot.writeBitSet(optionals, 3);
+        if (struct.isSetTinfo()) {
+          struct.tinfo.write(oprot);
+        }
+        if (struct.isSetCredentials()) {
+          struct.credentials.write(oprot);
+        }
+        if (struct.isSetDurability()) {
+          oprot.writeI32(struct.durability.getValue());
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, startUpdate_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.clientImpl.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.durability = org.apache.accumulo.core.tabletingest.thrift.TDurability.findByValue(iprot.readI32());
+          struct.setDurabilityIsSet(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();
+    }
+  }
+
+  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+  public static class startUpdate_result implements org.apache.thrift.TBase<startUpdate_result, startUpdate_result._Fields>, java.io.Serializable, Cloneable, Comparable<startUpdate_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("startUpdate_result");
+
+    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.I64, (short)0);
+    private static final org.apache.thrift.protocol.TField SEC_FIELD_DESC = new org.apache.thrift.protocol.TField("sec", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new startUpdate_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new startUpdate_resultTupleSchemeFactory();
+
+    public long success; // required
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec; // 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"),
+      SEC((short)1, "sec");
+
+      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: // SEC
+            return SEC;
+          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;
+      }
+
+      @Override
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      @Override
+      public java.lang.String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    private static final int __SUCCESS_ISSET_ID = 0;
+    private byte __isset_bitfield = 0;
+    public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64          , "UpdateID")));
+      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, org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException.class)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(startUpdate_result.class, metaDataMap);
+    }
+
+    public startUpdate_result() {
+    }
+
+    public startUpdate_result(
+      long success,
+      org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec)
+    {
+      this();
+      this.success = success;
+      setSuccessIsSet(true);
+      this.sec = sec;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public startUpdate_result(startUpdate_result other) {
+      __isset_bitfield = other.__isset_bitfield;
+      this.success = other.success;
+      if (other.isSetSec()) {
+        this.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException(other.sec);
+      }
+    }
+
+    @Override
+    public startUpdate_result deepCopy() {
+      return new startUpdate_result(this);
+    }
+
+    @Override
+    public void clear() {
+      setSuccessIsSet(false);
+      this.success = 0;
+      this.sec = null;
+    }
+
+    public long getSuccess() {
+      return this.success;
+    }
+
+    public startUpdate_result setSuccess(long success) {
+      this.success = success;
+      setSuccessIsSet(true);
+      return this;
+    }
+
+    public void unsetSuccess() {
+      __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __SUCCESS_ISSET_ID);
+    }
+
+    /** Returns true if field success is set (has been assigned a value) and false otherwise */
+    public boolean isSetSuccess() {
+      return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __SUCCESS_ISSET_ID);
+    }
+
+    public void setSuccessIsSet(boolean value) {
+      __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __SUCCESS_ISSET_ID, value);
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException getSec() {
+      return this.sec;
+    }
+
+    public startUpdate_result setSec(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.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;
+      }
+    }
+
+    @Override
+    public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+      switch (field) {
+      case SUCCESS:
+        if (value == null) {
+          unsetSuccess();
+        } else {
+          setSuccess((java.lang.Long)value);
+        }
+        break;
+
+      case SEC:
+        if (value == null) {
+          unsetSec();
+        } else {
+          setSec((org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException)value);
+        }
+        break;
+
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    @Override
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return getSuccess();
+
+      case SEC:
+        return getSec();
+
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    @Override
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case SUCCESS:
+        return isSetSuccess();
+      case SEC:
+        return isSetSec();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that instanceof startUpdate_result)
+        return this.equals((startUpdate_result)that);
+      return false;
+    }
+
+    public boolean equals(startUpdate_result that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return true;
+
+      boolean this_present_success = true;
+      boolean that_present_success = true;
+      if (this_present_success || that_present_success) {
+        if (!(this_present_success && that_present_success))
+          return false;
+        if (this.success != that.success)
+          return false;
+      }
+
+      boolean this_present_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;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(success);
+
+      hashCode = hashCode * 8191 + ((isSetSec()) ? 131071 : 524287);
+      if (isSetSec())
+        hashCode = hashCode * 8191 + sec.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(startUpdate_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(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;
+        }
+      }
+      return 0;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    @Override
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    @Override
+    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("startUpdate_result(");
+      boolean first = true;
+
+      sb.append("success:");
+      sb.append(this.success);
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("sec:");
+      if (this.sec == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.sec);
+      }
+      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 {
+        // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+        __isset_bitfield = 0;
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class startUpdate_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public startUpdate_resultStandardScheme getScheme() {
+        return new startUpdate_resultStandardScheme();
+      }
+    }
+
+    private static class startUpdate_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<startUpdate_result> {
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol iprot, startUpdate_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.I64) {
+                struct.success = iprot.readI64();
+                struct.setSuccessIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 1: // SEC
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException();
+                struct.sec.read(iprot);
+                struct.setSecIsSet(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();
+      }
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol oprot, startUpdate_result struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.isSetSuccess()) {
+          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+          oprot.writeI64(struct.success);
+          oprot.writeFieldEnd();
+        }
+        if (struct.sec != null) {
+          oprot.writeFieldBegin(SEC_FIELD_DESC);
+          struct.sec.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class startUpdate_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public startUpdate_resultTupleScheme getScheme() {
+        return new startUpdate_resultTupleScheme();
+      }
+    }
+
+    private static class startUpdate_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<startUpdate_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, startUpdate_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.isSetSec()) {
+          optionals.set(1);
+        }
+        oprot.writeBitSet(optionals, 2);
+        if (struct.isSetSuccess()) {
+          oprot.writeI64(struct.success);
+        }
+        if (struct.isSetSec()) {
+          struct.sec.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, startUpdate_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet incoming = iprot.readBitSet(2);
+        if (incoming.get(0)) {
+          struct.success = iprot.readI64();
+          struct.setSuccessIsSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException();
+          struct.sec.read(iprot);
+          struct.setSecIsSet(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();
+    }
+  }
+
+  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+  public static class applyUpdates_args implements org.apache.thrift.TBase<applyUpdates_args, applyUpdates_args._Fields>, java.io.Serializable, Cloneable, Comparable<applyUpdates_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("applyUpdates_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 UPDATE_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("updateID", org.apache.thrift.protocol.TType.I64, (short)2);
+    private static final org.apache.thrift.protocol.TField KEY_EXTENT_FIELD_DESC = new org.apache.thrift.protocol.TField("keyExtent", org.apache.thrift.protocol.TType.STRUCT, (short)3);
+    private static final org.apache.thrift.protocol.TField MUTATIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("mutations", org.apache.thrift.protocol.TType.LIST, (short)4);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new applyUpdates_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new applyUpdates_argsTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
+    public long updateID; // required
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent keyExtent; // required
+    public @org.apache.thrift.annotation.Nullable java.util.List<org.apache.accumulo.core.dataImpl.thrift.TMutation> mutations; // 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"),
+      UPDATE_ID((short)2, "updateID"),
+      KEY_EXTENT((short)3, "keyExtent"),
+      MUTATIONS((short)4, "mutations");
+
+      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: // UPDATE_ID
+            return UPDATE_ID;
+          case 3: // KEY_EXTENT
+            return KEY_EXTENT;
+          case 4: // MUTATIONS
+            return MUTATIONS;
+          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;
+      }
+
+      @Override
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      @Override
+      public java.lang.String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    private static final int __UPDATEID_ISSET_ID = 0;
+    private byte __isset_bitfield = 0;
+    public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.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.clientImpl.thrift.TInfo.class)));
+      tmpMap.put(_Fields.UPDATE_ID, new org.apache.thrift.meta_data.FieldMetaData("updateID", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64          , "UpdateID")));
+      tmpMap.put(_Fields.KEY_EXTENT, new org.apache.thrift.meta_data.FieldMetaData("keyExtent", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent.class)));
+      tmpMap.put(_Fields.MUTATIONS, new org.apache.thrift.meta_data.FieldMetaData("mutations", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+              new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.dataImpl.thrift.TMutation.class))));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(applyUpdates_args.class, metaDataMap);
+    }
+
+    public applyUpdates_args() {
+    }
+
+    public applyUpdates_args(
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
+      long updateID,
+      org.apache.accumulo.core.dataImpl.thrift.TKeyExtent keyExtent,
+      java.util.List<org.apache.accumulo.core.dataImpl.thrift.TMutation> mutations)
+    {
+      this();
+      this.tinfo = tinfo;
+      this.updateID = updateID;
+      setUpdateIDIsSet(true);
+      this.keyExtent = keyExtent;
+      this.mutations = mutations;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public applyUpdates_args(applyUpdates_args other) {
+      __isset_bitfield = other.__isset_bitfield;
+      if (other.isSetTinfo()) {
+        this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
+      }
+      this.updateID = other.updateID;
+      if (other.isSetKeyExtent()) {
+        this.keyExtent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(other.keyExtent);
+      }
+      if (other.isSetMutations()) {
+        java.util.List<org.apache.accumulo.core.dataImpl.thrift.TMutation> __this__mutations = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TMutation>(other.mutations.size());
+        for (org.apache.accumulo.core.dataImpl.thrift.TMutation other_element : other.mutations) {
+          __this__mutations.add(new org.apache.accumulo.core.dataImpl.thrift.TMutation(other_element));
+        }
+        this.mutations = __this__mutations;
+      }
+    }
+
+    @Override
+    public applyUpdates_args deepCopy() {
+      return new applyUpdates_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.tinfo = null;
+      setUpdateIDIsSet(false);
+      this.updateID = 0;
+      this.keyExtent = null;
+      this.mutations = null;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
+      return this.tinfo;
+    }
+
+    public applyUpdates_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.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;
+      }
+    }
+
+    public long getUpdateID() {
+      return this.updateID;
+    }
+
+    public applyUpdates_args setUpdateID(long updateID) {
+      this.updateID = updateID;
+      setUpdateIDIsSet(true);
+      return this;
+    }
+
+    public void unsetUpdateID() {
+      __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __UPDATEID_ISSET_ID);
+    }
+
+    /** Returns true if field updateID is set (has been assigned a value) and false otherwise */
+    public boolean isSetUpdateID() {
+      return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __UPDATEID_ISSET_ID);
+    }
+
+    public void setUpdateIDIsSet(boolean value) {
+      __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __UPDATEID_ISSET_ID, value);
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.dataImpl.thrift.TKeyExtent getKeyExtent() {
+      return this.keyExtent;
+    }
+
+    public applyUpdates_args setKeyExtent(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent keyExtent) {
+      this.keyExtent = keyExtent;
+      return this;
+    }
+
+    public void unsetKeyExtent() {
+      this.keyExtent = null;
+    }
+
+    /** Returns true if field keyExtent is set (has been assigned a value) and false otherwise */
+    public boolean isSetKeyExtent() {
+      return this.keyExtent != null;
+    }
+
+    public void setKeyExtentIsSet(boolean value) {
+      if (!value) {
+        this.keyExtent = null;
+      }
+    }
+
+    public int getMutationsSize() {
+      return (this.mutations == null) ? 0 : this.mutations.size();
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.util.Iterator<org.apache.accumulo.core.dataImpl.thrift.TMutation> getMutationsIterator() {
+      return (this.mutations == null) ? null : this.mutations.iterator();
+    }
+
+    public void addToMutations(org.apache.accumulo.core.dataImpl.thrift.TMutation elem) {
+      if (this.mutations == null) {
+        this.mutations = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TMutation>();
+      }
+      this.mutations.add(elem);
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.util.List<org.apache.accumulo.core.dataImpl.thrift.TMutation> getMutations() {
+      return this.mutations;
+    }
+
+    public applyUpdates_args setMutations(@org.apache.thrift.annotation.Nullable java.util.List<org.apache.accumulo.core.dataImpl.thrift.TMutation> mutations) {
+      this.mutations = mutations;
+      return this;
+    }
+
+    public void unsetMutations() {
+      this.mutations = null;
+    }
+
+    /** Returns true if field mutations is set (has been assigned a value) and false otherwise */
+    public boolean isSetMutations() {
+      return this.mutations != null;
+    }
+
+    public void setMutationsIsSet(boolean value) {
+      if (!value) {
+        this.mutations = null;
+      }
+    }
+
+    @Override
+    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.clientImpl.thrift.TInfo)value);
+        }
+        break;
+
+      case UPDATE_ID:
+        if (value == null) {
+          unsetUpdateID();
+        } else {
+          setUpdateID((java.lang.Long)value);
+        }
+        break;
+
+      case KEY_EXTENT:
+        if (value == null) {
+          unsetKeyExtent();
+        } else {
+          setKeyExtent((org.apache.accumulo.core.dataImpl.thrift.TKeyExtent)value);
+        }
+        break;
+
+      case MUTATIONS:
+        if (value == null) {
+          unsetMutations();
+        } else {
+          setMutations((java.util.List<org.apache.accumulo.core.dataImpl.thrift.TMutation>)value);
+        }
+        break;
+
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    @Override
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case TINFO:
+        return getTinfo();
+
+      case UPDATE_ID:
+        return getUpdateID();
+
+      case KEY_EXTENT:
+        return getKeyExtent();
+
+      case MUTATIONS:
+        return getMutations();
+
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    @Override
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case TINFO:
+        return isSetTinfo();
+      case UPDATE_ID:
+        return isSetUpdateID();
+      case KEY_EXTENT:
+        return isSetKeyExtent();
+      case MUTATIONS:
+        return isSetMutations();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that instanceof applyUpdates_args)
+        return this.equals((applyUpdates_args)that);
+      return false;
+    }
+
+    public boolean equals(applyUpdates_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_updateID = true;
+      boolean that_present_updateID = true;
+      if (this_present_updateID || that_present_updateID) {
+        if (!(this_present_updateID && that_present_updateID))
+          return false;
+        if (this.updateID != that.updateID)
+          return false;
+      }
+
+      boolean this_present_keyExtent = true && this.isSetKeyExtent();
+      boolean that_present_keyExtent = true && that.isSetKeyExtent();
+      if (this_present_keyExtent || that_present_keyExtent) {
+        if (!(this_present_keyExtent && that_present_keyExtent))
+          return false;
+        if (!this.keyExtent.equals(that.keyExtent))
+          return false;
+      }
+
+      boolean this_present_mutations = true && this.isSetMutations();
+      boolean that_present_mutations = true && that.isSetMutations();
+      if (this_present_mutations || that_present_mutations) {
+        if (!(this_present_mutations && that_present_mutations))
+          return false;
+        if (!this.mutations.equals(that.mutations))
+          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 + org.apache.thrift.TBaseHelper.hashCode(updateID);
+
+      hashCode = hashCode * 8191 + ((isSetKeyExtent()) ? 131071 : 524287);
+      if (isSetKeyExtent())
+        hashCode = hashCode * 8191 + keyExtent.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetMutations()) ? 131071 : 524287);
+      if (isSetMutations())
+        hashCode = hashCode * 8191 + mutations.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(applyUpdates_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(isSetUpdateID(), other.isSetUpdateID());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetUpdateID()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.updateID, other.updateID);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.compare(isSetKeyExtent(), other.isSetKeyExtent());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetKeyExtent()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.keyExtent, other.keyExtent);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.compare(isSetMutations(), other.isSetMutations());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetMutations()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.mutations, other.mutations);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    @Override
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      scheme(iprot).read(iprot, this);
+    }
+
+    @Override
+    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("applyUpdates_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("updateID:");
+      sb.append(this.updateID);
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("keyExtent:");
+      if (this.keyExtent == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.keyExtent);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("mutations:");
+      if (this.mutations == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.mutations);
+      }
+      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 (keyExtent != null) {
+        keyExtent.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 {
+        // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+        __isset_bitfield = 0;
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class applyUpdates_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public applyUpdates_argsStandardScheme getScheme() {
+        return new applyUpdates_argsStandardScheme();
+      }
+    }
+
+    private static class applyUpdates_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<applyUpdates_args> {
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol iprot, applyUpdates_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.clientImpl.thrift.TInfo();
+                struct.tinfo.read(iprot);
+                struct.setTinfoIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 2: // UPDATE_ID
+              if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+                struct.updateID = iprot.readI64();
+                struct.setUpdateIDIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 3: // KEY_EXTENT
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.keyExtent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
+                struct.keyExtent.read(iprot);
+                struct.setKeyExtentIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 4: // MUTATIONS
+              if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+                {
+                  org.apache.thrift.protocol.TList _list8 = iprot.readListBegin();
+                  struct.mutations = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TMutation>(_list8.size);
+                  @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TMutation _elem9;
+                  for (int _i10 = 0; _i10 < _list8.size; ++_i10)
+                  {
+                    _elem9 = new org.apache.accumulo.core.dataImpl.thrift.TMutation();
+                    _elem9.read(iprot);
+                    struct.mutations.add(_elem9);
+                  }
+                  iprot.readListEnd();
+                }
+                struct.setMutationsIsSet(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();
+      }
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol oprot, applyUpdates_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();
+        }
+        oprot.writeFieldBegin(UPDATE_ID_FIELD_DESC);
+        oprot.writeI64(struct.updateID);
+        oprot.writeFieldEnd();
+        if (struct.keyExtent != null) {
+          oprot.writeFieldBegin(KEY_EXTENT_FIELD_DESC);
+          struct.keyExtent.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.mutations != null) {
+          oprot.writeFieldBegin(MUTATIONS_FIELD_DESC);
+          {
+            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.mutations.size()));
+            for (org.apache.accumulo.core.dataImpl.thrift.TMutation _iter11 : struct.mutations)
+            {
+              _iter11.write(oprot);
+            }
+            oprot.writeListEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class applyUpdates_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public applyUpdates_argsTupleScheme getScheme() {
+        return new applyUpdates_argsTupleScheme();
+      }
+    }
+
+    private static class applyUpdates_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<applyUpdates_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, applyUpdates_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.isSetUpdateID()) {
+          optionals.set(1);
+        }
+        if (struct.isSetKeyExtent()) {
+          optionals.set(2);
+        }
+        if (struct.isSetMutations()) {
+          optionals.set(3);
+        }
+        oprot.writeBitSet(optionals, 4);
+        if (struct.isSetTinfo()) {
+          struct.tinfo.write(oprot);
+        }
+        if (struct.isSetUpdateID()) {
+          oprot.writeI64(struct.updateID);
+        }
+        if (struct.isSetKeyExtent()) {
+          struct.keyExtent.write(oprot);
+        }
+        if (struct.isSetMutations()) {
+          {
+            oprot.writeI32(struct.mutations.size());
+            for (org.apache.accumulo.core.dataImpl.thrift.TMutation _iter12 : struct.mutations)
+            {
+              _iter12.write(oprot);
+            }
+          }
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, applyUpdates_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(4);
+        if (incoming.get(0)) {
+          struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
+          struct.tinfo.read(iprot);
+          struct.setTinfoIsSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.updateID = iprot.readI64();
+          struct.setUpdateIDIsSet(true);
+        }
+        if (incoming.get(2)) {
+          struct.keyExtent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
+          struct.keyExtent.read(iprot);
+          struct.setKeyExtentIsSet(true);
+        }
+        if (incoming.get(3)) {
+          {
+            org.apache.thrift.protocol.TList _list13 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+            struct.mutations = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TMutation>(_list13.size);
+            @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TMutation _elem14;
+            for (int _i15 = 0; _i15 < _list13.size; ++_i15)
+            {
+              _elem14 = new org.apache.accumulo.core.dataImpl.thrift.TMutation();
+              _elem14.read(iprot);
+              struct.mutations.add(_elem14);
+            }
+          }
+          struct.setMutationsIsSet(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();
+    }
+  }
+
+  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+  public static class closeUpdate_args implements org.apache.thrift.TBase<closeUpdate_args, closeUpdate_args._Fields>, java.io.Serializable, Cloneable, Comparable<closeUpdate_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("closeUpdate_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 UPDATE_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("updateID", org.apache.thrift.protocol.TType.I64, (short)1);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new closeUpdate_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new closeUpdate_argsTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
+    public long updateID; // 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"),
+      UPDATE_ID((short)1, "updateID");
+
+      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 1: // UPDATE_ID
+            return UPDATE_ID;
+          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;
+      }
+
+      @Override
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      @Override
+      public java.lang.String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    private static final int __UPDATEID_ISSET_ID = 0;
+    private byte __isset_bitfield = 0;
+    public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.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.clientImpl.thrift.TInfo.class)));
+      tmpMap.put(_Fields.UPDATE_ID, new org.apache.thrift.meta_data.FieldMetaData("updateID", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64          , "UpdateID")));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(closeUpdate_args.class, metaDataMap);
+    }
+
+    public closeUpdate_args() {
+    }
+
+    public closeUpdate_args(
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
+      long updateID)
+    {
+      this();
+      this.tinfo = tinfo;
+      this.updateID = updateID;
+      setUpdateIDIsSet(true);
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public closeUpdate_args(closeUpdate_args other) {
+      __isset_bitfield = other.__isset_bitfield;
+      if (other.isSetTinfo()) {
+        this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
+      }
+      this.updateID = other.updateID;
+    }
+
+    @Override
+    public closeUpdate_args deepCopy() {
+      return new closeUpdate_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.tinfo = null;
+      setUpdateIDIsSet(false);
+      this.updateID = 0;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
+      return this.tinfo;
+    }
+
+    public closeUpdate_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.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;
+      }
+    }
+
+    public long getUpdateID() {
+      return this.updateID;
+    }
+
+    public closeUpdate_args setUpdateID(long updateID) {
+      this.updateID = updateID;
+      setUpdateIDIsSet(true);
+      return this;
+    }
+
+    public void unsetUpdateID() {
+      __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __UPDATEID_ISSET_ID);
+    }
+
+    /** Returns true if field updateID is set (has been assigned a value) and false otherwise */
+    public boolean isSetUpdateID() {
+      return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __UPDATEID_ISSET_ID);
+    }
+
+    public void setUpdateIDIsSet(boolean value) {
+      __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __UPDATEID_ISSET_ID, value);
+    }
+
+    @Override
+    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.clientImpl.thrift.TInfo)value);
+        }
+        break;
+
+      case UPDATE_ID:
+        if (value == null) {
+          unsetUpdateID();
+        } else {
+          setUpdateID((java.lang.Long)value);
+        }
+        break;
+
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    @Override
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case TINFO:
+        return getTinfo();
+
+      case UPDATE_ID:
+        return getUpdateID();
+
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    @Override
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case TINFO:
+        return isSetTinfo();
+      case UPDATE_ID:
+        return isSetUpdateID();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that instanceof closeUpdate_args)
+        return this.equals((closeUpdate_args)that);
+      return false;
+    }
+
+    public boolean equals(closeUpdate_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_updateID = true;
+      boolean that_present_updateID = true;
+      if (this_present_updateID || that_present_updateID) {
+        if (!(this_present_updateID && that_present_updateID))
+          return false;
+        if (this.updateID != that.updateID)
+          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 + org.apache.thrift.TBaseHelper.hashCode(updateID);
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(closeUpdate_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(isSetUpdateID(), other.isSetUpdateID());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetUpdateID()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.updateID, other.updateID);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    @Override
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      scheme(iprot).read(iprot, this);
+    }
+
+    @Override
+    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("closeUpdate_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("updateID:");
+      sb.append(this.updateID);
+      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();
+      }
+    }
+
+    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 {
+        // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+        __isset_bitfield = 0;
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class closeUpdate_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public closeUpdate_argsStandardScheme getScheme() {
+        return new closeUpdate_argsStandardScheme();
+      }
+    }
+
+    private static class closeUpdate_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<closeUpdate_args> {
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol iprot, closeUpdate_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.clientImpl.thrift.TInfo();
+                struct.tinfo.read(iprot);
+                struct.setTinfoIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 1: // UPDATE_ID
+              if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+                struct.updateID = iprot.readI64();
+                struct.setUpdateIDIsSet(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();
+      }
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol oprot, closeUpdate_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        oprot.writeFieldBegin(UPDATE_ID_FIELD_DESC);
+        oprot.writeI64(struct.updateID);
+        oprot.writeFieldEnd();
+        if (struct.tinfo != null) {
+          oprot.writeFieldBegin(TINFO_FIELD_DESC);
+          struct.tinfo.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class closeUpdate_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public closeUpdate_argsTupleScheme getScheme() {
+        return new closeUpdate_argsTupleScheme();
+      }
+    }
+
+    private static class closeUpdate_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<closeUpdate_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, closeUpdate_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.isSetUpdateID()) {
+          optionals.set(1);
+        }
+        oprot.writeBitSet(optionals, 2);
+        if (struct.isSetTinfo()) {
+          struct.tinfo.write(oprot);
+        }
+        if (struct.isSetUpdateID()) {
+          oprot.writeI64(struct.updateID);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, closeUpdate_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.clientImpl.thrift.TInfo();
+          struct.tinfo.read(iprot);
+          struct.setTinfoIsSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.updateID = iprot.readI64();
+          struct.setUpdateIDIsSet(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();
+    }
+  }
+
+  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+  public static class closeUpdate_result implements org.apache.thrift.TBase<closeUpdate_result, closeUpdate_result._Fields>, java.io.Serializable, Cloneable, Comparable<closeUpdate_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("closeUpdate_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 NSSI_FIELD_DESC = new org.apache.thrift.protocol.TField("nssi", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new closeUpdate_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new closeUpdate_resultTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.UpdateErrors success; // required
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException nssi; // 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"),
+      NSSI((short)1, "nssi");
+
+      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: // NSSI
+            return NSSI;
+          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;
+      }
+
+      @Override
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      @Override
+      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, org.apache.accumulo.core.dataImpl.thrift.UpdateErrors.class)));
+      tmpMap.put(_Fields.NSSI, new org.apache.thrift.meta_data.FieldMetaData("nssi", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException.class)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(closeUpdate_result.class, metaDataMap);
+    }
+
+    public closeUpdate_result() {
+    }
+
+    public closeUpdate_result(
+      org.apache.accumulo.core.dataImpl.thrift.UpdateErrors success,
+      org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException nssi)
+    {
+      this();
+      this.success = success;
+      this.nssi = nssi;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public closeUpdate_result(closeUpdate_result other) {
+      if (other.isSetSuccess()) {
+        this.success = new org.apache.accumulo.core.dataImpl.thrift.UpdateErrors(other.success);
+      }
+      if (other.isSetNssi()) {
+        this.nssi = new org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException(other.nssi);
+      }
+    }
+
+    @Override
+    public closeUpdate_result deepCopy() {
+      return new closeUpdate_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.success = null;
+      this.nssi = null;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.dataImpl.thrift.UpdateErrors getSuccess() {
+      return this.success;
+    }
+
+    public closeUpdate_result setSuccess(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.UpdateErrors 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 org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException getNssi() {
+      return this.nssi;
+    }
+
+    public closeUpdate_result setNssi(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException nssi) {
+      this.nssi = nssi;
+      return this;
+    }
+
+    public void unsetNssi() {
+      this.nssi = null;
+    }
+
+    /** Returns true if field nssi is set (has been assigned a value) and false otherwise */
+    public boolean isSetNssi() {
+      return this.nssi != null;
+    }
+
+    public void setNssiIsSet(boolean value) {
+      if (!value) {
+        this.nssi = null;
+      }
+    }
+
+    @Override
+    public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+      switch (field) {
+      case SUCCESS:
+        if (value == null) {
+          unsetSuccess();
+        } else {
+          setSuccess((org.apache.accumulo.core.dataImpl.thrift.UpdateErrors)value);
+        }
+        break;
+
+      case NSSI:
+        if (value == null) {
+          unsetNssi();
+        } else {
+          setNssi((org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException)value);
+        }
+        break;
+
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    @Override
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return getSuccess();
+
+      case NSSI:
+        return getNssi();
+
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    @Override
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case SUCCESS:
+        return isSetSuccess();
+      case NSSI:
+        return isSetNssi();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that instanceof closeUpdate_result)
+        return this.equals((closeUpdate_result)that);
+      return false;
+    }
+
+    public boolean equals(closeUpdate_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_nssi = true && this.isSetNssi();
+      boolean that_present_nssi = true && that.isSetNssi();
+      if (this_present_nssi || that_present_nssi) {
+        if (!(this_present_nssi && that_present_nssi))
+          return false;
+        if (!this.nssi.equals(that.nssi))
+          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 + ((isSetNssi()) ? 131071 : 524287);
+      if (isSetNssi())
+        hashCode = hashCode * 8191 + nssi.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(closeUpdate_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(isSetNssi(), other.isSetNssi());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetNssi()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nssi, other.nssi);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    @Override
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    @Override
+    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("closeUpdate_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("nssi:");
+      if (this.nssi == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.nssi);
+      }
+      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 closeUpdate_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public closeUpdate_resultStandardScheme getScheme() {
+        return new closeUpdate_resultStandardScheme();
+      }
+    }
+
+    private static class closeUpdate_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<closeUpdate_result> {
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol iprot, closeUpdate_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 org.apache.accumulo.core.dataImpl.thrift.UpdateErrors();
+                struct.success.read(iprot);
+                struct.setSuccessIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 1: // NSSI
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.nssi = new org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException();
+                struct.nssi.read(iprot);
+                struct.setNssiIsSet(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();
+      }
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol oprot, closeUpdate_result struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.success != null) {
+          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+          struct.success.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.nssi != null) {
+          oprot.writeFieldBegin(NSSI_FIELD_DESC);
+          struct.nssi.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class closeUpdate_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public closeUpdate_resultTupleScheme getScheme() {
+        return new closeUpdate_resultTupleScheme();
+      }
+    }
+
+    private static class closeUpdate_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<closeUpdate_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, closeUpdate_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.isSetNssi()) {
+          optionals.set(1);
+        }
+        oprot.writeBitSet(optionals, 2);
+        if (struct.isSetSuccess()) {
+          struct.success.write(oprot);
+        }
+        if (struct.isSetNssi()) {
+          struct.nssi.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, closeUpdate_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet incoming = iprot.readBitSet(2);
+        if (incoming.get(0)) {
+          struct.success = new org.apache.accumulo.core.dataImpl.thrift.UpdateErrors();
+          struct.success.read(iprot);
+          struct.setSuccessIsSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.nssi = new org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException();
+          struct.nssi.read(iprot);
+          struct.setNssiIsSet(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();
+    }
+  }
+
+  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+  public static class cancelUpdate_args implements org.apache.thrift.TBase<cancelUpdate_args, cancelUpdate_args._Fields>, java.io.Serializable, Cloneable, Comparable<cancelUpdate_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("cancelUpdate_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 UPDATE_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("updateID", org.apache.thrift.protocol.TType.I64, (short)2);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new cancelUpdate_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new cancelUpdate_argsTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
+    public long updateID; // 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"),
+      UPDATE_ID((short)2, "updateID");
+
+      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: // UPDATE_ID
+            return UPDATE_ID;
+          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;
+      }
+
+      @Override
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      @Override
+      public java.lang.String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    private static final int __UPDATEID_ISSET_ID = 0;
+    private byte __isset_bitfield = 0;
+    public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.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.clientImpl.thrift.TInfo.class)));
+      tmpMap.put(_Fields.UPDATE_ID, new org.apache.thrift.meta_data.FieldMetaData("updateID", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64          , "UpdateID")));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(cancelUpdate_args.class, metaDataMap);
+    }
+
+    public cancelUpdate_args() {
+    }
+
+    public cancelUpdate_args(
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
+      long updateID)
+    {
+      this();
+      this.tinfo = tinfo;
+      this.updateID = updateID;
+      setUpdateIDIsSet(true);
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public cancelUpdate_args(cancelUpdate_args other) {
+      __isset_bitfield = other.__isset_bitfield;
+      if (other.isSetTinfo()) {
+        this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
+      }
+      this.updateID = other.updateID;
+    }
+
+    @Override
+    public cancelUpdate_args deepCopy() {
+      return new cancelUpdate_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.tinfo = null;
+      setUpdateIDIsSet(false);
+      this.updateID = 0;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
+      return this.tinfo;
+    }
+
+    public cancelUpdate_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.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;
+      }
+    }
+
+    public long getUpdateID() {
+      return this.updateID;
+    }
+
+    public cancelUpdate_args setUpdateID(long updateID) {
+      this.updateID = updateID;
+      setUpdateIDIsSet(true);
+      return this;
+    }
+
+    public void unsetUpdateID() {
+      __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __UPDATEID_ISSET_ID);
+    }
+
+    /** Returns true if field updateID is set (has been assigned a value) and false otherwise */
+    public boolean isSetUpdateID() {
+      return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __UPDATEID_ISSET_ID);
+    }
+
+    public void setUpdateIDIsSet(boolean value) {
+      __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __UPDATEID_ISSET_ID, value);
+    }
+
+    @Override
+    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.clientImpl.thrift.TInfo)value);
+        }
+        break;
+
+      case UPDATE_ID:
+        if (value == null) {
+          unsetUpdateID();
+        } else {
+          setUpdateID((java.lang.Long)value);
+        }
+        break;
+
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    @Override
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case TINFO:
+        return getTinfo();
+
+      case UPDATE_ID:
+        return getUpdateID();
+
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    @Override
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case TINFO:
+        return isSetTinfo();
+      case UPDATE_ID:
+        return isSetUpdateID();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that instanceof cancelUpdate_args)
+        return this.equals((cancelUpdate_args)that);
+      return false;
+    }
+
+    public boolean equals(cancelUpdate_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_updateID = true;
+      boolean that_present_updateID = true;
+      if (this_present_updateID || that_present_updateID) {
+        if (!(this_present_updateID && that_present_updateID))
+          return false;
+        if (this.updateID != that.updateID)
+          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 + org.apache.thrift.TBaseHelper.hashCode(updateID);
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(cancelUpdate_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(isSetUpdateID(), other.isSetUpdateID());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetUpdateID()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.updateID, other.updateID);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    @Override
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      scheme(iprot).read(iprot, this);
+    }
+
+    @Override
+    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("cancelUpdate_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("updateID:");
+      sb.append(this.updateID);
+      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();
+      }
+    }
+
+    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 {
+        // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+        __isset_bitfield = 0;
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class cancelUpdate_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public cancelUpdate_argsStandardScheme getScheme() {
+        return new cancelUpdate_argsStandardScheme();
+      }
+    }
+
+    private static class cancelUpdate_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<cancelUpdate_args> {
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol iprot, cancelUpdate_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.clientImpl.thrift.TInfo();
+                struct.tinfo.read(iprot);
+                struct.setTinfoIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 2: // UPDATE_ID
+              if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+                struct.updateID = iprot.readI64();
+                struct.setUpdateIDIsSet(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();
+      }
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol oprot, cancelUpdate_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();
+        }
+        oprot.writeFieldBegin(UPDATE_ID_FIELD_DESC);
+        oprot.writeI64(struct.updateID);
+        oprot.writeFieldEnd();
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class cancelUpdate_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public cancelUpdate_argsTupleScheme getScheme() {
+        return new cancelUpdate_argsTupleScheme();
+      }
+    }
+
+    private static class cancelUpdate_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<cancelUpdate_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, cancelUpdate_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.isSetUpdateID()) {
+          optionals.set(1);
+        }
+        oprot.writeBitSet(optionals, 2);
+        if (struct.isSetTinfo()) {
+          struct.tinfo.write(oprot);
+        }
+        if (struct.isSetUpdateID()) {
+          oprot.writeI64(struct.updateID);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, cancelUpdate_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.clientImpl.thrift.TInfo();
+          struct.tinfo.read(iprot);
+          struct.setTinfoIsSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.updateID = iprot.readI64();
+          struct.setUpdateIDIsSet(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();
+    }
+  }
+
+  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+  public static class cancelUpdate_result implements org.apache.thrift.TBase<cancelUpdate_result, cancelUpdate_result._Fields>, java.io.Serializable, Cloneable, Comparable<cancelUpdate_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("cancelUpdate_result");
+
+    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.BOOL, (short)0);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new cancelUpdate_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new cancelUpdate_resultTupleSchemeFactory();
+
+    public boolean 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;
+      }
+
+      @Override
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      @Override
+      public java.lang.String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    private static final int __SUCCESS_ISSET_ID = 0;
+    private byte __isset_bitfield = 0;
+    public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(cancelUpdate_result.class, metaDataMap);
+    }
+
+    public cancelUpdate_result() {
+    }
+
+    public cancelUpdate_result(
+      boolean success)
+    {
+      this();
+      this.success = success;
+      setSuccessIsSet(true);
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public cancelUpdate_result(cancelUpdate_result other) {
+      __isset_bitfield = other.__isset_bitfield;
+      this.success = other.success;
+    }
+
+    @Override
+    public cancelUpdate_result deepCopy() {
+      return new cancelUpdate_result(this);
+    }
+
+    @Override
+    public void clear() {
+      setSuccessIsSet(false);
+      this.success = false;
+    }
+
+    public boolean isSuccess() {
+      return this.success;
+    }
+
+    public cancelUpdate_result setSuccess(boolean success) {
+      this.success = success;
+      setSuccessIsSet(true);
+      return this;
+    }
+
+    public void unsetSuccess() {
+      __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __SUCCESS_ISSET_ID);
+    }
+
+    /** Returns true if field success is set (has been assigned a value) and false otherwise */
+    public boolean isSetSuccess() {
+      return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __SUCCESS_ISSET_ID);
+    }
+
+    public void setSuccessIsSet(boolean value) {
+      __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __SUCCESS_ISSET_ID, value);
+    }
+
+    @Override
+    public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+      switch (field) {
+      case SUCCESS:
+        if (value == null) {
+          unsetSuccess();
+        } else {
+          setSuccess((java.lang.Boolean)value);
+        }
+        break;
+
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    @Override
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return isSuccess();
+
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    @Override
+    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 cancelUpdate_result)
+        return this.equals((cancelUpdate_result)that);
+      return false;
+    }
+
+    public boolean equals(cancelUpdate_result that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return true;
+
+      boolean this_present_success = true;
+      boolean that_present_success = true;
+      if (this_present_success || that_present_success) {
+        if (!(this_present_success && that_present_success))
+          return false;
+        if (this.success != that.success)
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      hashCode = hashCode * 8191 + ((success) ? 131071 : 524287);
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(cancelUpdate_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
+    @Override
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    @Override
+    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("cancelUpdate_result(");
+      boolean first = true;
+
+      sb.append("success:");
+      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 {
+        // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+        __isset_bitfield = 0;
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class cancelUpdate_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public cancelUpdate_resultStandardScheme getScheme() {
+        return new cancelUpdate_resultStandardScheme();
+      }
+    }
+
+    private static class cancelUpdate_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<cancelUpdate_result> {
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol iprot, cancelUpdate_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.BOOL) {
+                struct.success = iprot.readBool();
+                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();
+      }
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol oprot, cancelUpdate_result struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.isSetSuccess()) {
+          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+          oprot.writeBool(struct.success);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class cancelUpdate_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public cancelUpdate_resultTupleScheme getScheme() {
+        return new cancelUpdate_resultTupleScheme();
+      }
+    }
+
+    private static class cancelUpdate_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<cancelUpdate_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, cancelUpdate_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.writeBool(struct.success);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, cancelUpdate_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          struct.success = iprot.readBool();
+          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();
+    }
+  }
+
+  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+  public static class startConditionalUpdate_args implements org.apache.thrift.TBase<startConditionalUpdate_args, startConditionalUpdate_args._Fields>, java.io.Serializable, Cloneable, Comparable<startConditionalUpdate_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("startConditionalUpdate_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.protocol.TField AUTHORIZATIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("authorizations", org.apache.thrift.protocol.TType.LIST, (short)3);
+    private static final org.apache.thrift.protocol.TField TABLE_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("tableID", org.apache.thrift.protocol.TType.STRING, (short)4);
+    private static final org.apache.thrift.protocol.TField DURABILITY_FIELD_DESC = new org.apache.thrift.protocol.TField("durability", org.apache.thrift.protocol.TType.I32, (short)5);
+    private static final org.apache.thrift.protocol.TField CLASS_LOADER_CONTEXT_FIELD_DESC = new org.apache.thrift.protocol.TField("classLoaderContext", org.apache.thrift.protocol.TType.STRING, (short)6);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new startConditionalUpdate_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new startConditionalUpdate_argsTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.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.util.List<java.nio.ByteBuffer> authorizations; // required
+    public @org.apache.thrift.annotation.Nullable java.lang.String tableID; // required
+    /**
+     * 
+     * @see TDurability
+     */
+    public @org.apache.thrift.annotation.Nullable TDurability durability; // required
+    public @org.apache.thrift.annotation.Nullable java.lang.String classLoaderContext; // 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"),
+      AUTHORIZATIONS((short)3, "authorizations"),
+      TABLE_ID((short)4, "tableID"),
+      /**
+       * 
+       * @see TDurability
+       */
+      DURABILITY((short)5, "durability"),
+      CLASS_LOADER_CONTEXT((short)6, "classLoaderContext");
+
+      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;
+          case 3: // AUTHORIZATIONS
+            return AUTHORIZATIONS;
+          case 4: // TABLE_ID
+            return TABLE_ID;
+          case 5: // DURABILITY
+            return DURABILITY;
+          case 6: // CLASS_LOADER_CONTEXT
+            return CLASS_LOADER_CONTEXT;
+          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;
+      }
+
+      @Override
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      @Override
+      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.clientImpl.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.AUTHORIZATIONS, new org.apache.thrift.meta_data.FieldMetaData("authorizations", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+              new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING              , true))));
+      tmpMap.put(_Fields.TABLE_ID, new org.apache.thrift.meta_data.FieldMetaData("tableID", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      tmpMap.put(_Fields.DURABILITY, new org.apache.thrift.meta_data.FieldMetaData("durability", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TDurability.class)));
+      tmpMap.put(_Fields.CLASS_LOADER_CONTEXT, new org.apache.thrift.meta_data.FieldMetaData("classLoaderContext", 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(startConditionalUpdate_args.class, metaDataMap);
+    }
+
+    public startConditionalUpdate_args() {
+    }
+
+    public startConditionalUpdate_args(
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
+      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
+      java.util.List<java.nio.ByteBuffer> authorizations,
+      java.lang.String tableID,
+      TDurability durability,
+      java.lang.String classLoaderContext)
+    {
+      this();
+      this.tinfo = tinfo;
+      this.credentials = credentials;
+      this.authorizations = authorizations;
+      this.tableID = tableID;
+      this.durability = durability;
+      this.classLoaderContext = classLoaderContext;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public startConditionalUpdate_args(startConditionalUpdate_args other) {
+      if (other.isSetTinfo()) {
+        this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
+      }
+      if (other.isSetCredentials()) {
+        this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
+      }
+      if (other.isSetAuthorizations()) {
+        java.util.List<java.nio.ByteBuffer> __this__authorizations = new java.util.ArrayList<java.nio.ByteBuffer>(other.authorizations);
+        this.authorizations = __this__authorizations;
+      }
+      if (other.isSetTableID()) {
+        this.tableID = other.tableID;
+      }
+      if (other.isSetDurability()) {
+        this.durability = other.durability;
+      }
+      if (other.isSetClassLoaderContext()) {
+        this.classLoaderContext = other.classLoaderContext;
+      }
+    }
+
+    @Override
+    public startConditionalUpdate_args deepCopy() {
+      return new startConditionalUpdate_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.tinfo = null;
+      this.credentials = null;
+      this.authorizations = null;
+      this.tableID = null;
+      this.durability = null;
+      this.classLoaderContext = null;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
+      return this.tinfo;
+    }
+
+    public startConditionalUpdate_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.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 startConditionalUpdate_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 int getAuthorizationsSize() {
+      return (this.authorizations == null) ? 0 : this.authorizations.size();
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.util.Iterator<java.nio.ByteBuffer> getAuthorizationsIterator() {
+      return (this.authorizations == null) ? null : this.authorizations.iterator();
+    }
+
+    public void addToAuthorizations(java.nio.ByteBuffer elem) {
+      if (this.authorizations == null) {
+        this.authorizations = new java.util.ArrayList<java.nio.ByteBuffer>();
+      }
+      this.authorizations.add(elem);
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.util.List<java.nio.ByteBuffer> getAuthorizations() {
+      return this.authorizations;
+    }
+
+    public startConditionalUpdate_args setAuthorizations(@org.apache.thrift.annotation.Nullable java.util.List<java.nio.ByteBuffer> authorizations) {
+      this.authorizations = authorizations;
+      return this;
+    }
+
+    public void unsetAuthorizations() {
+      this.authorizations = null;
+    }
+
+    /** Returns true if field authorizations is set (has been assigned a value) and false otherwise */
+    public boolean isSetAuthorizations() {
+      return this.authorizations != null;
+    }
+
+    public void setAuthorizationsIsSet(boolean value) {
+      if (!value) {
+        this.authorizations = null;
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.lang.String getTableID() {
+      return this.tableID;
+    }
+
+    public startConditionalUpdate_args setTableID(@org.apache.thrift.annotation.Nullable java.lang.String tableID) {
+      this.tableID = tableID;
+      return this;
+    }
+
+    public void unsetTableID() {
+      this.tableID = null;
+    }
+
+    /** Returns true if field tableID is set (has been assigned a value) and false otherwise */
+    public boolean isSetTableID() {
+      return this.tableID != null;
+    }
+
+    public void setTableIDIsSet(boolean value) {
+      if (!value) {
+        this.tableID = null;
+      }
+    }
+
+    /**
+     * 
+     * @see TDurability
+     */
+    @org.apache.thrift.annotation.Nullable
+    public TDurability getDurability() {
+      return this.durability;
+    }
+
+    /**
+     * 
+     * @see TDurability
+     */
+    public startConditionalUpdate_args setDurability(@org.apache.thrift.annotation.Nullable TDurability durability) {
+      this.durability = durability;
+      return this;
+    }
+
+    public void unsetDurability() {
+      this.durability = null;
+    }
+
+    /** Returns true if field durability is set (has been assigned a value) and false otherwise */
+    public boolean isSetDurability() {
+      return this.durability != null;
+    }
+
+    public void setDurabilityIsSet(boolean value) {
+      if (!value) {
+        this.durability = null;
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.lang.String getClassLoaderContext() {
+      return this.classLoaderContext;
+    }
+
+    public startConditionalUpdate_args setClassLoaderContext(@org.apache.thrift.annotation.Nullable java.lang.String classLoaderContext) {
+      this.classLoaderContext = classLoaderContext;
+      return this;
+    }
+
+    public void unsetClassLoaderContext() {
+      this.classLoaderContext = null;
+    }
+
+    /** Returns true if field classLoaderContext is set (has been assigned a value) and false otherwise */
+    public boolean isSetClassLoaderContext() {
+      return this.classLoaderContext != null;
+    }
+
+    public void setClassLoaderContextIsSet(boolean value) {
+      if (!value) {
+        this.classLoaderContext = null;
+      }
+    }
+
+    @Override
+    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.clientImpl.thrift.TInfo)value);
+        }
+        break;
+
+      case CREDENTIALS:
+        if (value == null) {
+          unsetCredentials();
+        } else {
+          setCredentials((org.apache.accumulo.core.securityImpl.thrift.TCredentials)value);
+        }
+        break;
+
+      case AUTHORIZATIONS:
+        if (value == null) {
+          unsetAuthorizations();
+        } else {
+          setAuthorizations((java.util.List<java.nio.ByteBuffer>)value);
+        }
+        break;
+
+      case TABLE_ID:
+        if (value == null) {
+          unsetTableID();
+        } else {
+          setTableID((java.lang.String)value);
+        }
+        break;
+
+      case DURABILITY:
+        if (value == null) {
+          unsetDurability();
+        } else {
+          setDurability((TDurability)value);
+        }
+        break;
+
+      case CLASS_LOADER_CONTEXT:
+        if (value == null) {
+          unsetClassLoaderContext();
+        } else {
+          setClassLoaderContext((java.lang.String)value);
+        }
+        break;
+
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    @Override
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case TINFO:
+        return getTinfo();
+
+      case CREDENTIALS:
+        return getCredentials();
+
+      case AUTHORIZATIONS:
+        return getAuthorizations();
+
+      case TABLE_ID:
+        return getTableID();
+
+      case DURABILITY:
+        return getDurability();
+
+      case CLASS_LOADER_CONTEXT:
+        return getClassLoaderContext();
+
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    @Override
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case TINFO:
+        return isSetTinfo();
+      case CREDENTIALS:
+        return isSetCredentials();
+      case AUTHORIZATIONS:
+        return isSetAuthorizations();
+      case TABLE_ID:
+        return isSetTableID();
+      case DURABILITY:
+        return isSetDurability();
+      case CLASS_LOADER_CONTEXT:
+        return isSetClassLoaderContext();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that instanceof startConditionalUpdate_args)
+        return this.equals((startConditionalUpdate_args)that);
+      return false;
+    }
+
+    public boolean equals(startConditionalUpdate_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_authorizations = true && this.isSetAuthorizations();
+      boolean that_present_authorizations = true && that.isSetAuthorizations();
+      if (this_present_authorizations || that_present_authorizations) {
+        if (!(this_present_authorizations && that_present_authorizations))
+          return false;
+        if (!this.authorizations.equals(that.authorizations))
+          return false;
+      }
+
+      boolean this_present_tableID = true && this.isSetTableID();
+      boolean that_present_tableID = true && that.isSetTableID();
+      if (this_present_tableID || that_present_tableID) {
+        if (!(this_present_tableID && that_present_tableID))
+          return false;
+        if (!this.tableID.equals(that.tableID))
+          return false;
+      }
+
+      boolean this_present_durability = true && this.isSetDurability();
+      boolean that_present_durability = true && that.isSetDurability();
+      if (this_present_durability || that_present_durability) {
+        if (!(this_present_durability && that_present_durability))
+          return false;
+        if (!this.durability.equals(that.durability))
+          return false;
+      }
+
+      boolean this_present_classLoaderContext = true && this.isSetClassLoaderContext();
+      boolean that_present_classLoaderContext = true && that.isSetClassLoaderContext();
+      if (this_present_classLoaderContext || that_present_classLoaderContext) {
+        if (!(this_present_classLoaderContext && that_present_classLoaderContext))
+          return false;
+        if (!this.classLoaderContext.equals(that.classLoaderContext))
+          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 + ((isSetAuthorizations()) ? 131071 : 524287);
+      if (isSetAuthorizations())
+        hashCode = hashCode * 8191 + authorizations.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetTableID()) ? 131071 : 524287);
+      if (isSetTableID())
+        hashCode = hashCode * 8191 + tableID.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetDurability()) ? 131071 : 524287);
+      if (isSetDurability())
+        hashCode = hashCode * 8191 + durability.getValue();
+
+      hashCode = hashCode * 8191 + ((isSetClassLoaderContext()) ? 131071 : 524287);
+      if (isSetClassLoaderContext())
+        hashCode = hashCode * 8191 + classLoaderContext.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(startConditionalUpdate_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(isSetAuthorizations(), other.isSetAuthorizations());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetAuthorizations()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.authorizations, other.authorizations);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.compare(isSetTableID(), other.isSetTableID());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetTableID()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableID, other.tableID);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.compare(isSetDurability(), other.isSetDurability());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetDurability()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.durability, other.durability);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.compare(isSetClassLoaderContext(), other.isSetClassLoaderContext());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetClassLoaderContext()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.classLoaderContext, other.classLoaderContext);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    @Override
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      scheme(iprot).read(iprot, this);
+    }
+
+    @Override
+    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("startConditionalUpdate_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("authorizations:");
+      if (this.authorizations == null) {
+        sb.append("null");
+      } else {
+        org.apache.thrift.TBaseHelper.toString(this.authorizations, sb);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("tableID:");
+      if (this.tableID == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.tableID);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("durability:");
+      if (this.durability == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.durability);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("classLoaderContext:");
+      if (this.classLoaderContext == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.classLoaderContext);
+      }
+      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 startConditionalUpdate_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public startConditionalUpdate_argsStandardScheme getScheme() {
+        return new startConditionalUpdate_argsStandardScheme();
+      }
+    }
+
+    private static class startConditionalUpdate_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<startConditionalUpdate_args> {
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol iprot, startConditionalUpdate_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.clientImpl.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;
+            case 3: // AUTHORIZATIONS
+              if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+                {
+                  org.apache.thrift.protocol.TList _list16 = iprot.readListBegin();
+                  struct.authorizations = new java.util.ArrayList<java.nio.ByteBuffer>(_list16.size);
+                  @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem17;
+                  for (int _i18 = 0; _i18 < _list16.size; ++_i18)
+                  {
+                    _elem17 = iprot.readBinary();
+                    struct.authorizations.add(_elem17);
+                  }
+                  iprot.readListEnd();
+                }
+                struct.setAuthorizationsIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 4: // TABLE_ID
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.tableID = iprot.readString();
+                struct.setTableIDIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 5: // DURABILITY
+              if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+                struct.durability = org.apache.accumulo.core.tabletingest.thrift.TDurability.findByValue(iprot.readI32());
+                struct.setDurabilityIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 6: // CLASS_LOADER_CONTEXT
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.classLoaderContext = iprot.readString();
+                struct.setClassLoaderContextIsSet(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();
+      }
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol oprot, startConditionalUpdate_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();
+        }
+        if (struct.authorizations != null) {
+          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 _iter19 : struct.authorizations)
+            {
+              oprot.writeBinary(_iter19);
+            }
+            oprot.writeListEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+        if (struct.tableID != null) {
+          oprot.writeFieldBegin(TABLE_ID_FIELD_DESC);
+          oprot.writeString(struct.tableID);
+          oprot.writeFieldEnd();
+        }
+        if (struct.durability != null) {
+          oprot.writeFieldBegin(DURABILITY_FIELD_DESC);
+          oprot.writeI32(struct.durability.getValue());
+          oprot.writeFieldEnd();
+        }
+        if (struct.classLoaderContext != null) {
+          oprot.writeFieldBegin(CLASS_LOADER_CONTEXT_FIELD_DESC);
+          oprot.writeString(struct.classLoaderContext);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class startConditionalUpdate_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public startConditionalUpdate_argsTupleScheme getScheme() {
+        return new startConditionalUpdate_argsTupleScheme();
+      }
+    }
+
+    private static class startConditionalUpdate_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<startConditionalUpdate_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, startConditionalUpdate_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.isSetAuthorizations()) {
+          optionals.set(2);
+        }
+        if (struct.isSetTableID()) {
+          optionals.set(3);
+        }
+        if (struct.isSetDurability()) {
+          optionals.set(4);
+        }
+        if (struct.isSetClassLoaderContext()) {
+          optionals.set(5);
+        }
+        oprot.writeBitSet(optionals, 6);
+        if (struct.isSetTinfo()) {
+          struct.tinfo.write(oprot);
+        }
+        if (struct.isSetCredentials()) {
+          struct.credentials.write(oprot);
+        }
+        if (struct.isSetAuthorizations()) {
+          {
+            oprot.writeI32(struct.authorizations.size());
+            for (java.nio.ByteBuffer _iter20 : struct.authorizations)
+            {
+              oprot.writeBinary(_iter20);
+            }
+          }
+        }
+        if (struct.isSetTableID()) {
+          oprot.writeString(struct.tableID);
+        }
+        if (struct.isSetDurability()) {
+          oprot.writeI32(struct.durability.getValue());
+        }
+        if (struct.isSetClassLoaderContext()) {
+          oprot.writeString(struct.classLoaderContext);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, startConditionalUpdate_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(6);
+        if (incoming.get(0)) {
+          struct.tinfo = new org.apache.accumulo.core.clientImpl.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)) {
+          {
+            org.apache.thrift.protocol.TList _list21 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
+            struct.authorizations = new java.util.ArrayList<java.nio.ByteBuffer>(_list21.size);
+            @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem22;
+            for (int _i23 = 0; _i23 < _list21.size; ++_i23)
+            {
+              _elem22 = iprot.readBinary();
+              struct.authorizations.add(_elem22);
+            }
+          }
+          struct.setAuthorizationsIsSet(true);
+        }
+        if (incoming.get(3)) {
+          struct.tableID = iprot.readString();
+          struct.setTableIDIsSet(true);
+        }
+        if (incoming.get(4)) {
+          struct.durability = org.apache.accumulo.core.tabletingest.thrift.TDurability.findByValue(iprot.readI32());
+          struct.setDurabilityIsSet(true);
+        }
+        if (incoming.get(5)) {
+          struct.classLoaderContext = iprot.readString();
+          struct.setClassLoaderContextIsSet(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();
+    }
+  }
+
+  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+  public static class startConditionalUpdate_result implements org.apache.thrift.TBase<startConditionalUpdate_result, startConditionalUpdate_result._Fields>, java.io.Serializable, Cloneable, Comparable<startConditionalUpdate_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("startConditionalUpdate_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 SEC_FIELD_DESC = new org.apache.thrift.protocol.TField("sec", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new startConditionalUpdate_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new startConditionalUpdate_resultTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TConditionalSession success; // required
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec; // 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"),
+      SEC((short)1, "sec");
+
+      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: // SEC
+            return SEC;
+          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;
+      }
+
+      @Override
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      @Override
+      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, org.apache.accumulo.core.dataImpl.thrift.TConditionalSession.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, org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException.class)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(startConditionalUpdate_result.class, metaDataMap);
+    }
+
+    public startConditionalUpdate_result() {
+    }
+
+    public startConditionalUpdate_result(
+      org.apache.accumulo.core.dataImpl.thrift.TConditionalSession success,
+      org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec)
+    {
+      this();
+      this.success = success;
+      this.sec = sec;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public startConditionalUpdate_result(startConditionalUpdate_result other) {
+      if (other.isSetSuccess()) {
+        this.success = new org.apache.accumulo.core.dataImpl.thrift.TConditionalSession(other.success);
+      }
+      if (other.isSetSec()) {
+        this.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException(other.sec);
+      }
+    }
+
+    @Override
+    public startConditionalUpdate_result deepCopy() {
+      return new startConditionalUpdate_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.success = null;
+      this.sec = null;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.dataImpl.thrift.TConditionalSession getSuccess() {
+      return this.success;
+    }
+
+    public startConditionalUpdate_result setSuccess(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TConditionalSession 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 org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException getSec() {
+      return this.sec;
+    }
+
+    public startConditionalUpdate_result setSec(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.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;
+      }
+    }
+
+    @Override
+    public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+      switch (field) {
+      case SUCCESS:
+        if (value == null) {
+          unsetSuccess();
+        } else {
+          setSuccess((org.apache.accumulo.core.dataImpl.thrift.TConditionalSession)value);
+        }
+        break;
+
+      case SEC:
+        if (value == null) {
+          unsetSec();
+        } else {
+          setSec((org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException)value);
+        }
+        break;
+
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    @Override
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return getSuccess();
+
+      case SEC:
+        return getSec();
+
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    @Override
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case SUCCESS:
+        return isSetSuccess();
+      case SEC:
+        return isSetSec();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that instanceof startConditionalUpdate_result)
+        return this.equals((startConditionalUpdate_result)that);
+      return false;
+    }
+
+    public boolean equals(startConditionalUpdate_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_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;
+      }
+
+      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 + ((isSetSec()) ? 131071 : 524287);
+      if (isSetSec())
+        hashCode = hashCode * 8191 + sec.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(startConditionalUpdate_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(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;
+        }
+      }
+      return 0;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    @Override
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    @Override
+    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("startConditionalUpdate_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("sec:");
+      if (this.sec == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.sec);
+      }
+      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 startConditionalUpdate_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public startConditionalUpdate_resultStandardScheme getScheme() {
+        return new startConditionalUpdate_resultStandardScheme();
+      }
+    }
+
+    private static class startConditionalUpdate_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<startConditionalUpdate_result> {
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol iprot, startConditionalUpdate_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 org.apache.accumulo.core.dataImpl.thrift.TConditionalSession();
+                struct.success.read(iprot);
+                struct.setSuccessIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 1: // SEC
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException();
+                struct.sec.read(iprot);
+                struct.setSecIsSet(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();
+      }
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol oprot, startConditionalUpdate_result struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.success != null) {
+          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+          struct.success.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.sec != null) {
+          oprot.writeFieldBegin(SEC_FIELD_DESC);
+          struct.sec.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class startConditionalUpdate_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public startConditionalUpdate_resultTupleScheme getScheme() {
+        return new startConditionalUpdate_resultTupleScheme();
+      }
+    }
+
+    private static class startConditionalUpdate_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<startConditionalUpdate_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, startConditionalUpdate_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.isSetSec()) {
+          optionals.set(1);
+        }
+        oprot.writeBitSet(optionals, 2);
+        if (struct.isSetSuccess()) {
+          struct.success.write(oprot);
+        }
+        if (struct.isSetSec()) {
+          struct.sec.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, startConditionalUpdate_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet incoming = iprot.readBitSet(2);
+        if (incoming.get(0)) {
+          struct.success = new org.apache.accumulo.core.dataImpl.thrift.TConditionalSession();
+          struct.success.read(iprot);
+          struct.setSuccessIsSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException();
+          struct.sec.read(iprot);
+          struct.setSecIsSet(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();
+    }
+  }
+
+  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+  public static class conditionalUpdate_args implements org.apache.thrift.TBase<conditionalUpdate_args, conditionalUpdate_args._Fields>, java.io.Serializable, Cloneable, Comparable<conditionalUpdate_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("conditionalUpdate_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 SESS_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("sessID", org.apache.thrift.protocol.TType.I64, (short)2);
+    private static final org.apache.thrift.protocol.TField MUTATIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("mutations", org.apache.thrift.protocol.TType.MAP, (short)3);
+    private static final org.apache.thrift.protocol.TField SYMBOLS_FIELD_DESC = new org.apache.thrift.protocol.TField("symbols", org.apache.thrift.protocol.TType.LIST, (short)4);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new conditionalUpdate_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new conditionalUpdate_argsTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
+    public long sessID; // required
+    public @org.apache.thrift.annotation.Nullable java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>> mutations; // required
+    public @org.apache.thrift.annotation.Nullable java.util.List<java.lang.String> symbols; // 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"),
+      SESS_ID((short)2, "sessID"),
+      MUTATIONS((short)3, "mutations"),
+      SYMBOLS((short)4, "symbols");
+
+      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: // SESS_ID
+            return SESS_ID;
+          case 3: // MUTATIONS
+            return MUTATIONS;
+          case 4: // SYMBOLS
+            return SYMBOLS;
+          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;
+      }
+
+      @Override
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      @Override
+      public java.lang.String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    private static final int __SESSID_ISSET_ID = 0;
+    private byte __isset_bitfield = 0;
+    public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.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.clientImpl.thrift.TInfo.class)));
+      tmpMap.put(_Fields.SESS_ID, new org.apache.thrift.meta_data.FieldMetaData("sessID", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64          , "UpdateID")));
+      tmpMap.put(_Fields.MUTATIONS, new org.apache.thrift.meta_data.FieldMetaData("mutations", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.MAP          , "CMBatch")));
+      tmpMap.put(_Fields.SYMBOLS, new org.apache.thrift.meta_data.FieldMetaData("symbols", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+              new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(conditionalUpdate_args.class, metaDataMap);
+    }
+
+    public conditionalUpdate_args() {
+    }
+
+    public conditionalUpdate_args(
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
+      long sessID,
+      java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>> mutations,
+      java.util.List<java.lang.String> symbols)
+    {
+      this();
+      this.tinfo = tinfo;
+      this.sessID = sessID;
+      setSessIDIsSet(true);
+      this.mutations = mutations;
+      this.symbols = symbols;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public conditionalUpdate_args(conditionalUpdate_args other) {
+      __isset_bitfield = other.__isset_bitfield;
+      if (other.isSetTinfo()) {
+        this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
+      }
+      this.sessID = other.sessID;
+      if (other.isSetMutations()) {
+        java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>> __this__mutations = new java.util.HashMap<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>>(other.mutations.size());
+        for (java.util.Map.Entry<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>> other_element : other.mutations.entrySet()) {
+
+          org.apache.accumulo.core.dataImpl.thrift.TKeyExtent other_element_key = other_element.getKey();
+          java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation> other_element_value = other_element.getValue();
+
+          org.apache.accumulo.core.dataImpl.thrift.TKeyExtent __this__mutations_copy_key = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(other_element_key);
+
+          java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation> __this__mutations_copy_value = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>(other_element_value.size());
+          for (org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation other_element_value_element : other_element_value) {
+            __this__mutations_copy_value.add(new org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation(other_element_value_element));
+          }
+
+          __this__mutations.put(__this__mutations_copy_key, __this__mutations_copy_value);
+        }
+        this.mutations = __this__mutations;
+      }
+      if (other.isSetSymbols()) {
+        java.util.List<java.lang.String> __this__symbols = new java.util.ArrayList<java.lang.String>(other.symbols);
+        this.symbols = __this__symbols;
+      }
+    }
+
+    @Override
+    public conditionalUpdate_args deepCopy() {
+      return new conditionalUpdate_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.tinfo = null;
+      setSessIDIsSet(false);
+      this.sessID = 0;
+      this.mutations = null;
+      this.symbols = null;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
+      return this.tinfo;
+    }
+
+    public conditionalUpdate_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.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;
+      }
+    }
+
+    public long getSessID() {
+      return this.sessID;
+    }
+
+    public conditionalUpdate_args setSessID(long sessID) {
+      this.sessID = sessID;
+      setSessIDIsSet(true);
+      return this;
+    }
+
+    public void unsetSessID() {
+      __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __SESSID_ISSET_ID);
+    }
+
+    /** Returns true if field sessID is set (has been assigned a value) and false otherwise */
+    public boolean isSetSessID() {
+      return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __SESSID_ISSET_ID);
+    }
+
+    public void setSessIDIsSet(boolean value) {
+      __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __SESSID_ISSET_ID, value);
+    }
+
+    public int getMutationsSize() {
+      return (this.mutations == null) ? 0 : this.mutations.size();
+    }
+
+    public void putToMutations(org.apache.accumulo.core.dataImpl.thrift.TKeyExtent key, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation> val) {
+      if (this.mutations == null) {
+        this.mutations = new java.util.HashMap<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>>();
+      }
+      this.mutations.put(key, val);
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>> getMutations() {
+      return this.mutations;
+    }
+
+    public conditionalUpdate_args setMutations(@org.apache.thrift.annotation.Nullable java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>> mutations) {
+      this.mutations = mutations;
+      return this;
+    }
+
+    public void unsetMutations() {
+      this.mutations = null;
+    }
+
+    /** Returns true if field mutations is set (has been assigned a value) and false otherwise */
+    public boolean isSetMutations() {
+      return this.mutations != null;
+    }
+
+    public void setMutationsIsSet(boolean value) {
+      if (!value) {
+        this.mutations = null;
+      }
+    }
+
+    public int getSymbolsSize() {
+      return (this.symbols == null) ? 0 : this.symbols.size();
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.util.Iterator<java.lang.String> getSymbolsIterator() {
+      return (this.symbols == null) ? null : this.symbols.iterator();
+    }
+
+    public void addToSymbols(java.lang.String elem) {
+      if (this.symbols == null) {
+        this.symbols = new java.util.ArrayList<java.lang.String>();
+      }
+      this.symbols.add(elem);
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.util.List<java.lang.String> getSymbols() {
+      return this.symbols;
+    }
+
+    public conditionalUpdate_args setSymbols(@org.apache.thrift.annotation.Nullable java.util.List<java.lang.String> symbols) {
+      this.symbols = symbols;
+      return this;
+    }
+
+    public void unsetSymbols() {
+      this.symbols = null;
+    }
+
+    /** Returns true if field symbols is set (has been assigned a value) and false otherwise */
+    public boolean isSetSymbols() {
+      return this.symbols != null;
+    }
+
+    public void setSymbolsIsSet(boolean value) {
+      if (!value) {
+        this.symbols = null;
+      }
+    }
+
+    @Override
+    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.clientImpl.thrift.TInfo)value);
+        }
+        break;
+
+      case SESS_ID:
+        if (value == null) {
+          unsetSessID();
+        } else {
+          setSessID((java.lang.Long)value);
+        }
+        break;
+
+      case MUTATIONS:
+        if (value == null) {
+          unsetMutations();
+        } else {
+          setMutations((java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>>)value);
+        }
+        break;
+
+      case SYMBOLS:
+        if (value == null) {
+          unsetSymbols();
+        } else {
+          setSymbols((java.util.List<java.lang.String>)value);
+        }
+        break;
+
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    @Override
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case TINFO:
+        return getTinfo();
+
+      case SESS_ID:
+        return getSessID();
+
+      case MUTATIONS:
+        return getMutations();
+
+      case SYMBOLS:
+        return getSymbols();
+
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    @Override
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case TINFO:
+        return isSetTinfo();
+      case SESS_ID:
+        return isSetSessID();
+      case MUTATIONS:
+        return isSetMutations();
+      case SYMBOLS:
+        return isSetSymbols();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that instanceof conditionalUpdate_args)
+        return this.equals((conditionalUpdate_args)that);
+      return false;
+    }
+
+    public boolean equals(conditionalUpdate_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_sessID = true;
+      boolean that_present_sessID = true;
+      if (this_present_sessID || that_present_sessID) {
+        if (!(this_present_sessID && that_present_sessID))
+          return false;
+        if (this.sessID != that.sessID)
+          return false;
+      }
+
+      boolean this_present_mutations = true && this.isSetMutations();
+      boolean that_present_mutations = true && that.isSetMutations();
+      if (this_present_mutations || that_present_mutations) {
+        if (!(this_present_mutations && that_present_mutations))
+          return false;
+        if (!this.mutations.equals(that.mutations))
+          return false;
+      }
+
+      boolean this_present_symbols = true && this.isSetSymbols();
+      boolean that_present_symbols = true && that.isSetSymbols();
+      if (this_present_symbols || that_present_symbols) {
+        if (!(this_present_symbols && that_present_symbols))
+          return false;
+        if (!this.symbols.equals(that.symbols))
+          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 + org.apache.thrift.TBaseHelper.hashCode(sessID);
+
+      hashCode = hashCode * 8191 + ((isSetMutations()) ? 131071 : 524287);
+      if (isSetMutations())
+        hashCode = hashCode * 8191 + mutations.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetSymbols()) ? 131071 : 524287);
+      if (isSetSymbols())
+        hashCode = hashCode * 8191 + symbols.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(conditionalUpdate_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(isSetSessID(), other.isSetSessID());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetSessID()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessID, other.sessID);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.compare(isSetMutations(), other.isSetMutations());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetMutations()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.mutations, other.mutations);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.compare(isSetSymbols(), other.isSetSymbols());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetSymbols()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.symbols, other.symbols);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    @Override
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      scheme(iprot).read(iprot, this);
+    }
+
+    @Override
+    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("conditionalUpdate_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("sessID:");
+      sb.append(this.sessID);
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("mutations:");
+      if (this.mutations == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.mutations);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("symbols:");
+      if (this.symbols == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.symbols);
+      }
+      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();
+      }
+    }
+
+    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 {
+        // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+        __isset_bitfield = 0;
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class conditionalUpdate_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public conditionalUpdate_argsStandardScheme getScheme() {
+        return new conditionalUpdate_argsStandardScheme();
+      }
+    }
+
+    private static class conditionalUpdate_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<conditionalUpdate_args> {
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol iprot, conditionalUpdate_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.clientImpl.thrift.TInfo();
+                struct.tinfo.read(iprot);
+                struct.setTinfoIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 2: // SESS_ID
+              if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+                struct.sessID = iprot.readI64();
+                struct.setSessIDIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 3: // MUTATIONS
+              if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+                {
+                  org.apache.thrift.protocol.TMap _map24 = iprot.readMapBegin();
+                  struct.mutations = new java.util.HashMap<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>>(2*_map24.size);
+                  @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _key25;
+                  @org.apache.thrift.annotation.Nullable java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation> _val26;
+                  for (int _i27 = 0; _i27 < _map24.size; ++_i27)
+                  {
+                    _key25 = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
+                    _key25.read(iprot);
+                    {
+                      org.apache.thrift.protocol.TList _list28 = iprot.readListBegin();
+                      _val26 = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>(_list28.size);
+                      @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation _elem29;
+                      for (int _i30 = 0; _i30 < _list28.size; ++_i30)
+                      {
+                        _elem29 = new org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation();
+                        _elem29.read(iprot);
+                        _val26.add(_elem29);
+                      }
+                      iprot.readListEnd();
+                    }
+                    struct.mutations.put(_key25, _val26);
+                  }
+                  iprot.readMapEnd();
+                }
+                struct.setMutationsIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 4: // SYMBOLS
+              if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+                {
+                  org.apache.thrift.protocol.TList _list31 = iprot.readListBegin();
+                  struct.symbols = 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)
+                  {
+                    _elem32 = iprot.readString();
+                    struct.symbols.add(_elem32);
+                  }
+                  iprot.readListEnd();
+                }
+                struct.setSymbolsIsSet(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();
+      }
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol oprot, conditionalUpdate_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();
+        }
+        oprot.writeFieldBegin(SESS_ID_FIELD_DESC);
+        oprot.writeI64(struct.sessID);
+        oprot.writeFieldEnd();
+        if (struct.mutations != null) {
+          oprot.writeFieldBegin(MUTATIONS_FIELD_DESC);
+          {
+            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.LIST, struct.mutations.size()));
+            for (java.util.Map.Entry<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>> _iter34 : struct.mutations.entrySet())
+            {
+              _iter34.getKey().write(oprot);
+              {
+                oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, _iter34.getValue().size()));
+                for (org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation _iter35 : _iter34.getValue())
+                {
+                  _iter35.write(oprot);
+                }
+                oprot.writeListEnd();
+              }
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+        if (struct.symbols != null) {
+          oprot.writeFieldBegin(SYMBOLS_FIELD_DESC);
+          {
+            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.symbols.size()));
+            for (java.lang.String _iter36 : struct.symbols)
+            {
+              oprot.writeString(_iter36);
+            }
+            oprot.writeListEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class conditionalUpdate_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public conditionalUpdate_argsTupleScheme getScheme() {
+        return new conditionalUpdate_argsTupleScheme();
+      }
+    }
+
+    private static class conditionalUpdate_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<conditionalUpdate_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, conditionalUpdate_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.isSetSessID()) {
+          optionals.set(1);
+        }
+        if (struct.isSetMutations()) {
+          optionals.set(2);
+        }
+        if (struct.isSetSymbols()) {
+          optionals.set(3);
+        }
+        oprot.writeBitSet(optionals, 4);
+        if (struct.isSetTinfo()) {
+          struct.tinfo.write(oprot);
+        }
+        if (struct.isSetSessID()) {
+          oprot.writeI64(struct.sessID);
+        }
+        if (struct.isSetMutations()) {
+          {
+            oprot.writeI32(struct.mutations.size());
+            for (java.util.Map.Entry<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>> _iter37 : struct.mutations.entrySet())
+            {
+              _iter37.getKey().write(oprot);
+              {
+                oprot.writeI32(_iter37.getValue().size());
+                for (org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation _iter38 : _iter37.getValue())
+                {
+                  _iter38.write(oprot);
+                }
+              }
+            }
+          }
+        }
+        if (struct.isSetSymbols()) {
+          {
+            oprot.writeI32(struct.symbols.size());
+            for (java.lang.String _iter39 : struct.symbols)
+            {
+              oprot.writeString(_iter39);
+            }
+          }
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, conditionalUpdate_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(4);
+        if (incoming.get(0)) {
+          struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
+          struct.tinfo.read(iprot);
+          struct.setTinfoIsSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.sessID = iprot.readI64();
+          struct.setSessIDIsSet(true);
+        }
+        if (incoming.get(2)) {
+          {
+            org.apache.thrift.protocol.TMap _map40 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.LIST); 
+            struct.mutations = new java.util.HashMap<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>>(2*_map40.size);
+            @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _key41;
+            @org.apache.thrift.annotation.Nullable java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation> _val42;
+            for (int _i43 = 0; _i43 < _map40.size; ++_i43)
+            {
+              _key41 = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
+              _key41.read(iprot);
+              {
+                org.apache.thrift.protocol.TList _list44 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+                _val42 = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>(_list44.size);
+                @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation _elem45;
+                for (int _i46 = 0; _i46 < _list44.size; ++_i46)
+                {
+                  _elem45 = new org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation();
+                  _elem45.read(iprot);
+                  _val42.add(_elem45);
+                }
+              }
+              struct.mutations.put(_key41, _val42);
+            }
+          }
+          struct.setMutationsIsSet(true);
+        }
+        if (incoming.get(3)) {
+          {
+            org.apache.thrift.protocol.TList _list47 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
+            struct.symbols = new java.util.ArrayList<java.lang.String>(_list47.size);
+            @org.apache.thrift.annotation.Nullable java.lang.String _elem48;
+            for (int _i49 = 0; _i49 < _list47.size; ++_i49)
+            {
+              _elem48 = iprot.readString();
+              struct.symbols.add(_elem48);
+            }
+          }
+          struct.setSymbolsIsSet(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();
+    }
+  }
+
+  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+  public static class conditionalUpdate_result implements org.apache.thrift.TBase<conditionalUpdate_result, conditionalUpdate_result._Fields>, java.io.Serializable, Cloneable, Comparable<conditionalUpdate_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("conditionalUpdate_result");
+
+    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.LIST, (short)0);
+    private static final org.apache.thrift.protocol.TField NSSI_FIELD_DESC = new org.apache.thrift.protocol.TField("nssi", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new conditionalUpdate_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new conditionalUpdate_resultTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable java.util.List<org.apache.accumulo.core.dataImpl.thrift.TCMResult> success; // required
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException nssi; // 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"),
+      NSSI((short)1, "nssi");
+
+      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: // NSSI
+            return NSSI;
+          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;
+      }
+
+      @Override
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      @Override
+      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.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+              new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.dataImpl.thrift.TCMResult.class))));
+      tmpMap.put(_Fields.NSSI, new org.apache.thrift.meta_data.FieldMetaData("nssi", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException.class)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(conditionalUpdate_result.class, metaDataMap);
+    }
+
+    public conditionalUpdate_result() {
+    }
+
+    public conditionalUpdate_result(
+      java.util.List<org.apache.accumulo.core.dataImpl.thrift.TCMResult> success,
+      org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException nssi)
+    {
+      this();
+      this.success = success;
+      this.nssi = nssi;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public conditionalUpdate_result(conditionalUpdate_result other) {
+      if (other.isSetSuccess()) {
+        java.util.List<org.apache.accumulo.core.dataImpl.thrift.TCMResult> __this__success = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TCMResult>(other.success.size());
+        for (org.apache.accumulo.core.dataImpl.thrift.TCMResult other_element : other.success) {
+          __this__success.add(new org.apache.accumulo.core.dataImpl.thrift.TCMResult(other_element));
+        }
+        this.success = __this__success;
+      }
+      if (other.isSetNssi()) {
+        this.nssi = new org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException(other.nssi);
+      }
+    }
+
+    @Override
+    public conditionalUpdate_result deepCopy() {
+      return new conditionalUpdate_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.success = null;
+      this.nssi = null;
+    }
+
+    public int getSuccessSize() {
+      return (this.success == null) ? 0 : this.success.size();
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.util.Iterator<org.apache.accumulo.core.dataImpl.thrift.TCMResult> getSuccessIterator() {
+      return (this.success == null) ? null : this.success.iterator();
+    }
+
+    public void addToSuccess(org.apache.accumulo.core.dataImpl.thrift.TCMResult elem) {
+      if (this.success == null) {
+        this.success = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TCMResult>();
+      }
+      this.success.add(elem);
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.util.List<org.apache.accumulo.core.dataImpl.thrift.TCMResult> getSuccess() {
+      return this.success;
+    }
+
+    public conditionalUpdate_result setSuccess(@org.apache.thrift.annotation.Nullable java.util.List<org.apache.accumulo.core.dataImpl.thrift.TCMResult> 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 org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException getNssi() {
+      return this.nssi;
+    }
+
+    public conditionalUpdate_result setNssi(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException nssi) {
+      this.nssi = nssi;
+      return this;
+    }
+
+    public void unsetNssi() {
+      this.nssi = null;
+    }
+
+    /** Returns true if field nssi is set (has been assigned a value) and false otherwise */
+    public boolean isSetNssi() {
+      return this.nssi != null;
+    }
+
+    public void setNssiIsSet(boolean value) {
+      if (!value) {
+        this.nssi = null;
+      }
+    }
+
+    @Override
+    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.List<org.apache.accumulo.core.dataImpl.thrift.TCMResult>)value);
+        }
+        break;
+
+      case NSSI:
+        if (value == null) {
+          unsetNssi();
+        } else {
+          setNssi((org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException)value);
+        }
+        break;
+
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    @Override
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return getSuccess();
+
+      case NSSI:
+        return getNssi();
+
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    @Override
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case SUCCESS:
+        return isSetSuccess();
+      case NSSI:
+        return isSetNssi();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that instanceof conditionalUpdate_result)
+        return this.equals((conditionalUpdate_result)that);
+      return false;
+    }
+
+    public boolean equals(conditionalUpdate_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_nssi = true && this.isSetNssi();
+      boolean that_present_nssi = true && that.isSetNssi();
+      if (this_present_nssi || that_present_nssi) {
+        if (!(this_present_nssi && that_present_nssi))
+          return false;
+        if (!this.nssi.equals(that.nssi))
+          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 + ((isSetNssi()) ? 131071 : 524287);
+      if (isSetNssi())
+        hashCode = hashCode * 8191 + nssi.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(conditionalUpdate_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(isSetNssi(), other.isSetNssi());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetNssi()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nssi, other.nssi);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    @Override
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    @Override
+    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("conditionalUpdate_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("nssi:");
+      if (this.nssi == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.nssi);
+      }
+      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 conditionalUpdate_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public conditionalUpdate_resultStandardScheme getScheme() {
+        return new conditionalUpdate_resultStandardScheme();
+      }
+    }
+
+    private static class conditionalUpdate_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<conditionalUpdate_result> {
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol iprot, conditionalUpdate_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.LIST) {
+                {
+                  org.apache.thrift.protocol.TList _list50 = iprot.readListBegin();
+                  struct.success = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TCMResult>(_list50.size);
+                  @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TCMResult _elem51;
+                  for (int _i52 = 0; _i52 < _list50.size; ++_i52)
+                  {
+                    _elem51 = new org.apache.accumulo.core.dataImpl.thrift.TCMResult();
+                    _elem51.read(iprot);
+                    struct.success.add(_elem51);
+                  }
+                  iprot.readListEnd();
+                }
+                struct.setSuccessIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 1: // NSSI
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.nssi = new org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException();
+                struct.nssi.read(iprot);
+                struct.setNssiIsSet(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();
+      }
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol oprot, conditionalUpdate_result struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.success != null) {
+          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+          {
+            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
+            for (org.apache.accumulo.core.dataImpl.thrift.TCMResult _iter53 : struct.success)
+            {
+              _iter53.write(oprot);
+            }
+            oprot.writeListEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+        if (struct.nssi != null) {
+          oprot.writeFieldBegin(NSSI_FIELD_DESC);
+          struct.nssi.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class conditionalUpdate_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public conditionalUpdate_resultTupleScheme getScheme() {
+        return new conditionalUpdate_resultTupleScheme();
+      }
+    }
+
+    private static class conditionalUpdate_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<conditionalUpdate_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, conditionalUpdate_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.isSetNssi()) {
+          optionals.set(1);
+        }
+        oprot.writeBitSet(optionals, 2);
+        if (struct.isSetSuccess()) {
+          {
+            oprot.writeI32(struct.success.size());
+            for (org.apache.accumulo.core.dataImpl.thrift.TCMResult _iter54 : struct.success)
+            {
+              _iter54.write(oprot);
+            }
+          }
+        }
+        if (struct.isSetNssi()) {
+          struct.nssi.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, conditionalUpdate_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.TList _list55 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+            struct.success = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TCMResult>(_list55.size);
+            @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TCMResult _elem56;
+            for (int _i57 = 0; _i57 < _list55.size; ++_i57)
+            {
+              _elem56 = new org.apache.accumulo.core.dataImpl.thrift.TCMResult();
+              _elem56.read(iprot);
+              struct.success.add(_elem56);
+            }
+          }
+          struct.setSuccessIsSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.nssi = new org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException();
+          struct.nssi.read(iprot);
+          struct.setNssiIsSet(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();
+    }
+  }
+
+  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+  public static class invalidateConditionalUpdate_args implements org.apache.thrift.TBase<invalidateConditionalUpdate_args, invalidateConditionalUpdate_args._Fields>, java.io.Serializable, Cloneable, Comparable<invalidateConditionalUpdate_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("invalidateConditionalUpdate_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 SESS_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("sessID", org.apache.thrift.protocol.TType.I64, (short)2);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new invalidateConditionalUpdate_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new invalidateConditionalUpdate_argsTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
+    public long sessID; // 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"),
+      SESS_ID((short)2, "sessID");
+
+      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: // SESS_ID
+            return SESS_ID;
+          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;
+      }
+
+      @Override
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      @Override
+      public java.lang.String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    private static final int __SESSID_ISSET_ID = 0;
+    private byte __isset_bitfield = 0;
+    public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.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.clientImpl.thrift.TInfo.class)));
+      tmpMap.put(_Fields.SESS_ID, new org.apache.thrift.meta_data.FieldMetaData("sessID", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64          , "UpdateID")));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(invalidateConditionalUpdate_args.class, metaDataMap);
+    }
+
+    public invalidateConditionalUpdate_args() {
+    }
+
+    public invalidateConditionalUpdate_args(
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
+      long sessID)
+    {
+      this();
+      this.tinfo = tinfo;
+      this.sessID = sessID;
+      setSessIDIsSet(true);
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public invalidateConditionalUpdate_args(invalidateConditionalUpdate_args other) {
+      __isset_bitfield = other.__isset_bitfield;
+      if (other.isSetTinfo()) {
+        this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
+      }
+      this.sessID = other.sessID;
+    }
+
+    @Override
+    public invalidateConditionalUpdate_args deepCopy() {
+      return new invalidateConditionalUpdate_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.tinfo = null;
+      setSessIDIsSet(false);
+      this.sessID = 0;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
+      return this.tinfo;
+    }
+
+    public invalidateConditionalUpdate_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.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;
+      }
+    }
+
+    public long getSessID() {
+      return this.sessID;
+    }
+
+    public invalidateConditionalUpdate_args setSessID(long sessID) {
+      this.sessID = sessID;
+      setSessIDIsSet(true);
+      return this;
+    }
+
+    public void unsetSessID() {
+      __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __SESSID_ISSET_ID);
+    }
+
+    /** Returns true if field sessID is set (has been assigned a value) and false otherwise */
+    public boolean isSetSessID() {
+      return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __SESSID_ISSET_ID);
+    }
+
+    public void setSessIDIsSet(boolean value) {
+      __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __SESSID_ISSET_ID, value);
+    }
+
+    @Override
+    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.clientImpl.thrift.TInfo)value);
+        }
+        break;
+
+      case SESS_ID:
+        if (value == null) {
+          unsetSessID();
+        } else {
+          setSessID((java.lang.Long)value);
+        }
+        break;
+
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    @Override
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case TINFO:
+        return getTinfo();
+
+      case SESS_ID:
+        return getSessID();
+
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    @Override
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case TINFO:
+        return isSetTinfo();
+      case SESS_ID:
+        return isSetSessID();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that instanceof invalidateConditionalUpdate_args)
+        return this.equals((invalidateConditionalUpdate_args)that);
+      return false;
+    }
+
+    public boolean equals(invalidateConditionalUpdate_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_sessID = true;
+      boolean that_present_sessID = true;
+      if (this_present_sessID || that_present_sessID) {
+        if (!(this_present_sessID && that_present_sessID))
+          return false;
+        if (this.sessID != that.sessID)
+          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 + org.apache.thrift.TBaseHelper.hashCode(sessID);
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(invalidateConditionalUpdate_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(isSetSessID(), other.isSetSessID());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetSessID()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessID, other.sessID);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    @Override
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      scheme(iprot).read(iprot, this);
+    }
+
+    @Override
+    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("invalidateConditionalUpdate_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("sessID:");
+      sb.append(this.sessID);
+      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();
+      }
+    }
+
+    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 {
+        // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+        __isset_bitfield = 0;
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class invalidateConditionalUpdate_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public invalidateConditionalUpdate_argsStandardScheme getScheme() {
+        return new invalidateConditionalUpdate_argsStandardScheme();
+      }
+    }
+
+    private static class invalidateConditionalUpdate_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<invalidateConditionalUpdate_args> {
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol iprot, invalidateConditionalUpdate_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.clientImpl.thrift.TInfo();
+                struct.tinfo.read(iprot);
+                struct.setTinfoIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 2: // SESS_ID
+              if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+                struct.sessID = iprot.readI64();
+                struct.setSessIDIsSet(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();
+      }
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol oprot, invalidateConditionalUpdate_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();
+        }
+        oprot.writeFieldBegin(SESS_ID_FIELD_DESC);
+        oprot.writeI64(struct.sessID);
+        oprot.writeFieldEnd();
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class invalidateConditionalUpdate_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public invalidateConditionalUpdate_argsTupleScheme getScheme() {
+        return new invalidateConditionalUpdate_argsTupleScheme();
+      }
+    }
+
+    private static class invalidateConditionalUpdate_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<invalidateConditionalUpdate_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, invalidateConditionalUpdate_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.isSetSessID()) {
+          optionals.set(1);
+        }
+        oprot.writeBitSet(optionals, 2);
+        if (struct.isSetTinfo()) {
+          struct.tinfo.write(oprot);
+        }
+        if (struct.isSetSessID()) {
+          oprot.writeI64(struct.sessID);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, invalidateConditionalUpdate_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.clientImpl.thrift.TInfo();
+          struct.tinfo.read(iprot);
+          struct.setTinfoIsSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.sessID = iprot.readI64();
+          struct.setSessIDIsSet(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();
+    }
+  }
+
+  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+  public static class invalidateConditionalUpdate_result implements org.apache.thrift.TBase<invalidateConditionalUpdate_result, invalidateConditionalUpdate_result._Fields>, java.io.Serializable, Cloneable, Comparable<invalidateConditionalUpdate_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("invalidateConditionalUpdate_result");
+
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new invalidateConditionalUpdate_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new invalidateConditionalUpdate_resultTupleSchemeFactory();
+
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+;
+
+      private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
+
+      static {
+        for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      @org.apache.thrift.annotation.Nullable
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      @org.apache.thrift.annotation.Nullable
+      public static _Fields findByName(java.lang.String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final java.lang.String _fieldName;
+
+      _Fields(short thriftId, java.lang.String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      @Override
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      @Override
+      public java.lang.String getFieldName() {
+        return _fieldName;
+      }
+    }
+    public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(invalidateConditionalUpdate_result.class, metaDataMap);
+    }
+
+    public invalidateConditionalUpdate_result() {
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public invalidateConditionalUpdate_result(invalidateConditionalUpdate_result other) {
+    }
+
+    @Override
+    public invalidateConditionalUpdate_result deepCopy() {
+      return new invalidateConditionalUpdate_result(this);
+    }
+
+    @Override
+    public void clear() {
+    }
+
+    @Override
+    public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+      switch (field) {
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    @Override
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    @Override
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that instanceof invalidateConditionalUpdate_result)
+        return this.equals((invalidateConditionalUpdate_result)that);
+      return false;
+    }
+
+    public boolean equals(invalidateConditionalUpdate_result that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return true;
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(invalidateConditionalUpdate_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      return 0;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    @Override
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    @Override
+    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("invalidateConditionalUpdate_result(");
+      boolean first = true;
+
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class invalidateConditionalUpdate_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public invalidateConditionalUpdate_resultStandardScheme getScheme() {
+        return new invalidateConditionalUpdate_resultStandardScheme();
+      }
+    }
+
+    private static class invalidateConditionalUpdate_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<invalidateConditionalUpdate_result> {
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol iprot, invalidateConditionalUpdate_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) {
+            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();
+      }
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol oprot, invalidateConditionalUpdate_result struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class invalidateConditionalUpdate_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public invalidateConditionalUpdate_resultTupleScheme getScheme() {
+        return new invalidateConditionalUpdate_resultTupleScheme();
+      }
+    }
+
+    private static class invalidateConditionalUpdate_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<invalidateConditionalUpdate_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, invalidateConditionalUpdate_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, invalidateConditionalUpdate_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+      }
+    }
+
+    private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
+      return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
+    }
+  }
+
+  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+  public static class closeConditionalUpdate_args implements org.apache.thrift.TBase<closeConditionalUpdate_args, closeConditionalUpdate_args._Fields>, java.io.Serializable, Cloneable, Comparable<closeConditionalUpdate_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("closeConditionalUpdate_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 SESS_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("sessID", org.apache.thrift.protocol.TType.I64, (short)2);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new closeConditionalUpdate_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new closeConditionalUpdate_argsTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
+    public long sessID; // 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"),
+      SESS_ID((short)2, "sessID");
+
+      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: // SESS_ID
+            return SESS_ID;
+          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;
+      }
+
+      @Override
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      @Override
+      public java.lang.String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    private static final int __SESSID_ISSET_ID = 0;
+    private byte __isset_bitfield = 0;
+    public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.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.clientImpl.thrift.TInfo.class)));
+      tmpMap.put(_Fields.SESS_ID, new org.apache.thrift.meta_data.FieldMetaData("sessID", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64          , "UpdateID")));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(closeConditionalUpdate_args.class, metaDataMap);
+    }
+
+    public closeConditionalUpdate_args() {
+    }
+
+    public closeConditionalUpdate_args(
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
+      long sessID)
+    {
+      this();
+      this.tinfo = tinfo;
+      this.sessID = sessID;
+      setSessIDIsSet(true);
+    }
+